[KYUUBI #1647] Implement GetTableTypes operation

<!--
Thanks for sending a pull request!

Here are some tips for you:
  1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html
  2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
  3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
-->

### _Why are the changes needed?_
<!--
Please clarify why the changes are needed. For instance,
  1. If you add a feature, you can talk about the use case of it.
  2. If you fix a bug, you can clarify why it is a bug.
-->

### _How was this patch tested?_
- [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible

- [ ] Add screenshots for manual tests if appropriate

- [ ] [Run test](https://kyuubi.readthedocs.io/en/latest/develop_tools/testing.html#running-tests) locally before make a pull request

Closes #1686 from yanghua/KYUUBI-1647.

Closes #1647

c25f5caa [yanghua] retrigger ci
52ae1a8b [yanghua] [KYUUBI #1647] Implement GetTableTypes operation

Authored-by: yanghua <yanghua1127@gmail.com>
Signed-off-by: Cheng Pan <chengpan@apache.org>
This commit is contained in:
yanghua 2022-01-07 09:39:37 +08:00 committed by Cheng Pan
parent 628719fd60
commit ce7916c3f4
No known key found for this signature in database
GPG Key ID: 8001952629BCC75D
3 changed files with 54 additions and 1 deletions

View File

@ -52,7 +52,10 @@ class FlinkSQLOperationManager extends OperationManager("FlinkSQLOperationManage
tableName: String,
tableTypes: util.List[String]): Operation = null
override def newGetTableTypesOperation(session: Session): Operation = null
override def newGetTableTypesOperation(session: Session): Operation = {
val op = new GetTableTypes(session)
addOperation(op)
}
override def newGetColumnsOperation(
session: Session,

View File

@ -0,0 +1,35 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kyuubi.engine.flink.operation
import scala.collection.JavaConverters.seqAsJavaListConverter
import org.apache.kyuubi.engine.flink.result.{Constants, OperationUtil}
import org.apache.kyuubi.operation.OperationType
import org.apache.kyuubi.operation.meta.ResultSetSchemaConstant.TABLE_TYPE
import org.apache.kyuubi.session.Session
class GetTableTypes(session: Session)
extends FlinkOperation(OperationType.GET_TABLE_TYPES, session) {
override protected def runInternal(): Unit = {
resultSet = OperationUtil.stringListToResultSet(
Constants.SUPPORTED_TABLE_TYPES.toList.asJava,
TABLE_TYPE)
}
}

View File

@ -21,7 +21,9 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout
import org.scalatest.time.SpanSugar._
import org.apache.kyuubi.engine.flink.WithFlinkSQLEngine
import org.apache.kyuubi.engine.flink.result.Constants
import org.apache.kyuubi.operation.HiveJDBCTestHelper
import org.apache.kyuubi.operation.meta.ResultSetSchemaConstant.TABLE_TYPE
import org.apache.kyuubi.service.ServiceState._
class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
@ -52,6 +54,19 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
}
}
test("get table type for flink sql") {
withJdbcStatement() { statement =>
val meta = statement.getConnection.getMetaData
val types = meta.getTableTypes
val expected = Constants.SUPPORTED_TABLE_TYPES.toIterator
while (types.next()) {
assert(types.getString(TABLE_TYPE) === expected.next())
}
assert(!expected.hasNext)
assert(!types.next())
}
}
test("execute statement - select column name with dots") {
withJdbcStatement() { statement =>
val resultSet = statement.executeQuery("select 'tmp.hello'")