-
Notifications
You must be signed in to change notification settings - Fork 935
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[KYUUBI #5851] Generalize TRowSet generators
# 🔍 Description ## Issue References 🔗 As described. ## Describe Your Solution 🔧 - Introduced a generalized RowSet generator `AbstractTRowSetGenerator[SchemaT, RowT, ColumnT]` - extract common methods for looping and assembling the rows to TRowSet - support generation for either column-based or row-based TRowSet - Each engine creates a sub-generator of `AbstractTRowSetGenerator` - focus on mapping and conversion from the engine's data type to the relative Thrift type - implements the schema data type and column value methods - create a generator instance instead of the previously used `RowSet` object, for isolated session-aware or thread-aware configs or context, eg. Timezone ID for Flink, and the Hive time formatters for Spark. - This PR covers the TRowSet generation for the server and the engines of Spark/Flink/Trino/Chat, except the JDBC engine which will be supported in the follow-ups with JDBC dialect support. ## Types of changes 🔖 - [ ] Bugfix (non-breaking change which fixes an issue) - [ ] New feature (non-breaking change which adds functionality) - [ ] Breaking change (fix or feature that would cause existing functionality to change) ## Test Plan 🧪 #### Behavior Without This Pull Request ⚰️ No behavior changes. #### Behavior With This Pull Request 🎉 No behavior changes. #### Related Unit Tests CI tests. --- # Checklists ## 📝 Author Self Checklist - [x] My code follows the [style guidelines](https://kyuubi.readthedocs.io/en/master/contributing/code/style.html) of this project - [x] I have performed a self-review - [x] I have commented my code, particularly in hard-to-understand areas - [ ] I have made corresponding changes to the documentation - [ ] My changes generate no new warnings - [ ] I have added tests that prove my fix is effective or that my feature works - [x] New and existing unit tests pass locally with my changes - [x] This patch was not authored or co-authored using [Generative Tooling](https://www.apache.org/legal/generative-tooling.html) ## 📝 Committer Pre-Merge Checklist - [ ] Pull request title is okay. - [ ] No license issues. - [ ] Milestone correctly set? - [ ] Test coverage is ok - [ ] Assignees are selected. - [ ] Minimum number of approvals - [ ] No changes are requested **Be nice. Be informative.** Closes #5851 from bowenliang123/rowset-gen. Closes #5851 1d2f73a [Bowen Liang] common RowSetGenerator Authored-by: Bowen Liang <[email protected]> Signed-off-by: Bowen Liang <[email protected]>
1 parent
64ee629
commit 9047151
Showing
21 changed files
with
785 additions
and
1,028 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
52 changes: 52 additions & 0 deletions
52
...hat-engine/src/main/scala/org/apache/kyuubi/engine/chat/schema/ChatTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,52 @@ | ||
/* | ||
* 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.chat.schema | ||
|
||
import org.apache.kyuubi.engine.chat.schema.ChatTRowSetGenerator._ | ||
import org.apache.kyuubi.engine.schema.AbstractTRowSetGenerator | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TTypeId._ | ||
|
||
class ChatTRowSetGenerator | ||
extends AbstractTRowSetGenerator[Seq[String], Seq[String], String] { | ||
|
||
override def getColumnSizeFromSchemaType(schema: Seq[String]): Int = schema.length | ||
|
||
override def getColumnType(schema: Seq[String], ordinal: Int): String = COL_STRING_TYPE | ||
|
||
override protected def isColumnNullAt(row: Seq[String], ordinal: Int): Boolean = | ||
row(ordinal) == null | ||
|
||
override def getColumnAs[T](row: Seq[String], ordinal: Int): T = row(ordinal).asInstanceOf[T] | ||
|
||
override def toTColumn(rows: Seq[Seq[String]], ordinal: Int, typ: String): TColumn = | ||
typ match { | ||
case COL_STRING_TYPE => toTTypeColumn(STRING_TYPE, rows, ordinal) | ||
case otherType => throw new UnsupportedOperationException(s"type $otherType") | ||
} | ||
|
||
override def toTColumnValue(ordinal: Int, row: Seq[String], types: Seq[String]): TColumnValue = | ||
getColumnType(types, ordinal) match { | ||
case "String" => toTTypeColumnVal(STRING_TYPE, row, ordinal) | ||
case otherType => throw new UnsupportedOperationException(s"type $otherType") | ||
} | ||
} | ||
|
||
object ChatTRowSetGenerator { | ||
val COL_STRING_TYPE: String = classOf[String].getSimpleName | ||
} |
107 changes: 0 additions & 107 deletions
107
...rnals/kyuubi-chat-engine/src/main/scala/org/apache/kyuubi/engine/chat/schema/RowSet.scala
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
141 changes: 141 additions & 0 deletions
141
...l-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/FlinkTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,141 @@ | ||
/* | ||
* 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.schema | ||
|
||
import java.time.{Instant, ZonedDateTime, ZoneId} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.flink.table.data.StringData | ||
import org.apache.flink.table.types.logical._ | ||
import org.apache.flink.types.Row | ||
|
||
import org.apache.kyuubi.engine.flink.result.ResultSet | ||
import org.apache.kyuubi.engine.flink.schema.RowSet.{toHiveString, TIMESTAMP_LZT_FORMATTER} | ||
import org.apache.kyuubi.engine.schema.AbstractTRowSetGenerator | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TTypeId._ | ||
import org.apache.kyuubi.util.RowSetUtils.bitSetToBuffer | ||
|
||
class FlinkTRowSetGenerator(zoneId: ZoneId) | ||
extends AbstractTRowSetGenerator[ResultSet, Row, LogicalType] { | ||
override def getColumnSizeFromSchemaType(schema: ResultSet): Int = schema.columns.size | ||
|
||
override def getColumnType(schema: ResultSet, ordinal: Int): LogicalType = | ||
schema.columns.get(ordinal).getDataType.getLogicalType | ||
|
||
override def isColumnNullAt(row: Row, ordinal: Int): Boolean = row.getField(ordinal) == null | ||
|
||
override def getColumnAs[T](row: Row, ordinal: Int): T = row.getFieldAs[T](ordinal) | ||
|
||
override def toTColumnValue(ordinal: Int, row: Row, types: ResultSet): TColumnValue = { | ||
getColumnType(types, ordinal) match { | ||
case _: BooleanType => toTTypeColumnVal(BOOLEAN_TYPE, row, ordinal) | ||
case _: TinyIntType => toTTypeColumnVal(BINARY_TYPE, row, ordinal) | ||
case _: SmallIntType => toTTypeColumnVal(TINYINT_TYPE, row, ordinal) | ||
case _: IntType => toTTypeColumnVal(INT_TYPE, row, ordinal) | ||
case _: BigIntType => toTTypeColumnVal(BIGINT_TYPE, row, ordinal) | ||
case _: DoubleType => toTTypeColumnVal(DOUBLE_TYPE, row, ordinal) | ||
case _: FloatType => toTTypeColumnVal(FLOAT_TYPE, row, ordinal) | ||
case t @ (_: VarCharType | _: CharType) => | ||
val tStringValue = new TStringValue | ||
val fieldValue = row.getField(ordinal) | ||
fieldValue match { | ||
case value: String => | ||
tStringValue.setValue(value) | ||
case value: StringData => | ||
tStringValue.setValue(value.toString) | ||
case null => | ||
tStringValue.setValue(null) | ||
case other => | ||
throw new IllegalArgumentException( | ||
s"Unsupported conversion class ${other.getClass} " + | ||
s"for type ${t.getClass}.") | ||
} | ||
TColumnValue.stringVal(tStringValue) | ||
case _: LocalZonedTimestampType => | ||
val tStringValue = new TStringValue | ||
val fieldValue = row.getField(ordinal) | ||
tStringValue.setValue(TIMESTAMP_LZT_FORMATTER.format( | ||
ZonedDateTime.ofInstant(fieldValue.asInstanceOf[Instant], zoneId))) | ||
TColumnValue.stringVal(tStringValue) | ||
case t => | ||
val tStringValue = new TStringValue | ||
if (row.getField(ordinal) != null) { | ||
tStringValue.setValue(toHiveString((row.getField(ordinal), t))) | ||
} | ||
TColumnValue.stringVal(tStringValue) | ||
} | ||
} | ||
|
||
override def toTColumn(rows: Seq[Row], ordinal: Int, logicalType: LogicalType): TColumn = { | ||
val nulls = new java.util.BitSet() | ||
// for each column, determine the conversion class by sampling the first non-value value | ||
// if there's no row, set the entire column empty | ||
val sampleField = rows.iterator.map(_.getField(ordinal)).find(_ ne null).orNull | ||
logicalType match { | ||
case _: BooleanType => toTTypeColumn(BOOLEAN_TYPE, rows, ordinal) | ||
case _: TinyIntType => toTTypeColumn(BINARY_TYPE, rows, ordinal) | ||
case _: SmallIntType => toTTypeColumn(TINYINT_TYPE, rows, ordinal) | ||
case _: IntType => toTTypeColumn(INT_TYPE, rows, ordinal) | ||
case _: BigIntType => toTTypeColumn(BIGINT_TYPE, rows, ordinal) | ||
case _: FloatType => toTTypeColumn(FLOAT_TYPE, rows, ordinal) | ||
case _: DoubleType => toTTypeColumn(DOUBLE_TYPE, rows, ordinal) | ||
case t @ (_: VarCharType | _: CharType) => | ||
val values: java.util.List[String] = new java.util.ArrayList[String](0) | ||
sampleField match { | ||
case _: String => | ||
values.addAll(getOrSetAsNull[String](rows, ordinal, nulls, "")) | ||
case _: StringData => | ||
val stringDataValues = | ||
getOrSetAsNull[StringData](rows, ordinal, nulls, StringData.fromString("")) | ||
stringDataValues.forEach(e => values.add(e.toString)) | ||
case null => | ||
values.addAll(getOrSetAsNull[String](rows, ordinal, nulls, "")) | ||
case other => | ||
throw new IllegalArgumentException( | ||
s"Unsupported conversion class ${other.getClass} " + | ||
s"for type ${t.getClass}.") | ||
} | ||
TColumn.stringVal(new TStringColumn(values, nulls)) | ||
case _: LocalZonedTimestampType => | ||
val values = getOrSetAsNull[Instant](rows, ordinal, nulls, Instant.EPOCH) | ||
.toArray().map(v => | ||
TIMESTAMP_LZT_FORMATTER.format( | ||
ZonedDateTime.ofInstant(v.asInstanceOf[Instant], zoneId))) | ||
TColumn.stringVal(new TStringColumn(values.toList.asJava, nulls)) | ||
case _ => | ||
var i = 0 | ||
val rowSize = rows.length | ||
val values = new java.util.ArrayList[String](rowSize) | ||
while (i < rowSize) { | ||
val row = rows(i) | ||
nulls.set(i, row.getField(ordinal) == null) | ||
val value = | ||
if (row.getField(ordinal) == null) { | ||
"" | ||
} else { | ||
toHiveString((row.getField(ordinal), logicalType)) | ||
} | ||
values.add(value) | ||
i += 1 | ||
} | ||
TColumn.stringVal(new TStringColumn(values, nulls)) | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
77 changes: 77 additions & 0 deletions
77
...ine/src/main/scala/org/apache/kyuubi/engine/spark/schema/SparkArrowTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,77 @@ | ||
/* | ||
* 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.spark.schema | ||
|
||
import java.nio.ByteBuffer | ||
|
||
import org.apache.spark.sql.types._ | ||
|
||
import org.apache.kyuubi.engine.schema.AbstractTRowSetGenerator | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.util.RowSetUtils.bitSetToBuffer | ||
|
||
class SparkArrowTRowSetGenerator | ||
extends AbstractTRowSetGenerator[StructType, Array[Byte], DataType] { | ||
override def toColumnBasedSet(rows: Seq[Array[Byte]], schema: StructType): TRowSet = { | ||
require(schema.length == 1, "ArrowRowSetGenerator accepts only one single byte array") | ||
require(schema.head.dataType == BinaryType, "ArrowRowSetGenerator accepts only BinaryType") | ||
|
||
val tRowSet = new TRowSet(0, new java.util.ArrayList[TRow](1)) | ||
val tColumn = toTColumn(rows, 1, schema.head.dataType) | ||
tRowSet.addToColumns(tColumn) | ||
tRowSet | ||
} | ||
|
||
override def toTColumn(rows: Seq[Array[Byte]], ordinal: Int, typ: DataType): TColumn = { | ||
require(rows.length == 1, "ArrowRowSetGenerator accepts only one single byte array") | ||
typ match { | ||
case BinaryType => | ||
val values = new java.util.ArrayList[ByteBuffer](1) | ||
values.add(ByteBuffer.wrap(rows.head)) | ||
val nulls = new java.util.BitSet() | ||
TColumn.binaryVal(new TBinaryColumn(values, nulls)) | ||
case _ => throw new IllegalArgumentException( | ||
s"unsupported datatype $typ, ArrowRowSetGenerator accepts only BinaryType") | ||
} | ||
} | ||
|
||
override def toRowBasedSet(rows: Seq[Array[Byte]], schema: StructType): TRowSet = { | ||
throw new UnsupportedOperationException | ||
} | ||
|
||
override def getColumnSizeFromSchemaType(schema: StructType): Int = { | ||
throw new UnsupportedOperationException | ||
} | ||
|
||
override def getColumnType(schema: StructType, ordinal: Int): DataType = { | ||
throw new UnsupportedOperationException | ||
} | ||
|
||
override def isColumnNullAt(row: Array[Byte], ordinal: Int): Boolean = { | ||
throw new UnsupportedOperationException | ||
} | ||
|
||
override def getColumnAs[T](row: Array[Byte], ordinal: Int): T = { | ||
throw new UnsupportedOperationException | ||
} | ||
|
||
override def toTColumnValue(ordinal: Int, row: Array[Byte], types: StructType): TColumnValue = { | ||
throw new UnsupportedOperationException | ||
} | ||
|
||
} |
93 changes: 93 additions & 0 deletions
93
...l-engine/src/main/scala/org/apache/kyuubi/engine/spark/schema/SparkTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
/* | ||
* 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.spark.schema | ||
|
||
import org.apache.spark.sql.Row | ||
import org.apache.spark.sql.execution.HiveResult | ||
import org.apache.spark.sql.execution.HiveResult.TimeFormatters | ||
import org.apache.spark.sql.types._ | ||
|
||
import org.apache.kyuubi.engine.schema.AbstractTRowSetGenerator | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TTypeId._ | ||
import org.apache.kyuubi.util.RowSetUtils.bitSetToBuffer | ||
|
||
class SparkTRowSetGenerator | ||
extends AbstractTRowSetGenerator[StructType, Row, DataType] { | ||
|
||
// reused time formatters in single RowSet generation, see KYUUBI-5811 | ||
private val tf = HiveResult.getTimeFormatters | ||
|
||
override def getColumnSizeFromSchemaType(schema: StructType): Int = schema.length | ||
|
||
override def getColumnType(schema: StructType, ordinal: Int): DataType = schema(ordinal).dataType | ||
|
||
override def isColumnNullAt(row: Row, ordinal: Int): Boolean = row.isNullAt(ordinal) | ||
|
||
override def getColumnAs[T](row: Row, ordinal: Int): T = row.getAs[T](ordinal) | ||
|
||
override def toTColumn(rows: Seq[Row], ordinal: Int, typ: DataType): TColumn = { | ||
val timeFormatters: TimeFormatters = tf | ||
val nulls = new java.util.BitSet() | ||
typ match { | ||
case BooleanType => toTTypeColumn(BOOLEAN_TYPE, rows, ordinal) | ||
case ByteType => toTTypeColumn(BINARY_TYPE, rows, ordinal) | ||
case ShortType => toTTypeColumn(TINYINT_TYPE, rows, ordinal) | ||
case IntegerType => toTTypeColumn(INT_TYPE, rows, ordinal) | ||
case LongType => toTTypeColumn(BIGINT_TYPE, rows, ordinal) | ||
case FloatType => toTTypeColumn(FLOAT_TYPE, rows, ordinal) | ||
case DoubleType => toTTypeColumn(DOUBLE_TYPE, rows, ordinal) | ||
case StringType => toTTypeColumn(STRING_TYPE, rows, ordinal) | ||
case BinaryType => toTTypeColumn(ARRAY_TYPE, rows, ordinal) | ||
case _ => | ||
var i = 0 | ||
val rowSize = rows.length | ||
val values = new java.util.ArrayList[String](rowSize) | ||
while (i < rowSize) { | ||
val row = rows(i) | ||
nulls.set(i, row.isNullAt(ordinal)) | ||
values.add(RowSet.toHiveString(row.get(ordinal) -> typ, timeFormatters = timeFormatters)) | ||
i += 1 | ||
} | ||
TColumn.stringVal(new TStringColumn(values, nulls)) | ||
} | ||
} | ||
|
||
override def toTColumnValue(ordinal: Int, row: Row, types: StructType): TColumnValue = { | ||
val timeFormatters: TimeFormatters = tf | ||
getColumnType(types, ordinal) match { | ||
case BooleanType => toTTypeColumnVal(BOOLEAN_TYPE, row, ordinal) | ||
case ByteType => toTTypeColumnVal(BINARY_TYPE, row, ordinal) | ||
case ShortType => toTTypeColumnVal(TINYINT_TYPE, row, ordinal) | ||
case IntegerType => toTTypeColumnVal(INT_TYPE, row, ordinal) | ||
case LongType => toTTypeColumnVal(BIGINT_TYPE, row, ordinal) | ||
case FloatType => toTTypeColumnVal(FLOAT_TYPE, row, ordinal) | ||
case DoubleType => toTTypeColumnVal(DOUBLE_TYPE, row, ordinal) | ||
case StringType => toTTypeColumnVal(STRING_TYPE, row, ordinal) | ||
case _ => | ||
val tStrValue = new TStringValue | ||
if (!row.isNullAt(ordinal)) { | ||
tStrValue.setValue(RowSet.toHiveString( | ||
row.get(ordinal) -> types(ordinal).dataType, | ||
timeFormatters = timeFormatters)) | ||
} | ||
TColumnValue.stringVal(tStrValue) | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
96 changes: 96 additions & 0 deletions
96
...o-engine/src/main/scala/org/apache/kyuubi/engine/trino/schema/TrinoTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,96 @@ | ||
/* | ||
* 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.trino.schema | ||
|
||
import io.trino.client.{ClientTypeSignature, Column} | ||
import io.trino.client.ClientStandardTypes._ | ||
|
||
import org.apache.kyuubi.engine.schema.AbstractTRowSetGenerator | ||
import org.apache.kyuubi.engine.trino.schema.RowSet.toHiveString | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TTypeId._ | ||
import org.apache.kyuubi.util.RowSetUtils._ | ||
|
||
class TrinoTRowSetGenerator | ||
extends AbstractTRowSetGenerator[Seq[Column], Seq[_], ClientTypeSignature] { | ||
|
||
override def getColumnSizeFromSchemaType(schema: Seq[Column]): Int = schema.length | ||
|
||
override def getColumnType(schema: Seq[Column], ordinal: Int): ClientTypeSignature = { | ||
schema(ordinal).getTypeSignature | ||
} | ||
|
||
override def isColumnNullAt(row: Seq[_], ordinal: Int): Boolean = | ||
row(ordinal) == null | ||
|
||
override def getColumnAs[T](row: Seq[_], ordinal: Int): T = | ||
row(ordinal).asInstanceOf[T] | ||
|
||
override def toTColumn(rows: Seq[Seq[_]], ordinal: Int, typ: ClientTypeSignature): TColumn = { | ||
val nulls = new java.util.BitSet() | ||
typ.getRawType match { | ||
case BOOLEAN => toTTypeColumn(BOOLEAN_TYPE, rows, ordinal) | ||
case TINYINT => toTTypeColumn(BINARY_TYPE, rows, ordinal) | ||
case SMALLINT => toTTypeColumn(TINYINT_TYPE, rows, ordinal) | ||
case INTEGER => toTTypeColumn(INT_TYPE, rows, ordinal) | ||
case BIGINT => toTTypeColumn(BIGINT_TYPE, rows, ordinal) | ||
case REAL => toTTypeColumn(FLOAT_TYPE, rows, ordinal) | ||
case DOUBLE => toTTypeColumn(DOUBLE_TYPE, rows, ordinal) | ||
case VARCHAR => toTTypeColumn(STRING_TYPE, rows, ordinal) | ||
case VARBINARY => toTTypeColumn(ARRAY_TYPE, rows, ordinal) | ||
case _ => | ||
val rowSize = rows.length | ||
val values = new java.util.ArrayList[String](rowSize) | ||
var i = 0 | ||
while (i < rowSize) { | ||
val row = rows(i) | ||
val isNull = isColumnNullAt(row, ordinal) | ||
nulls.set(i, isNull) | ||
val value = if (isNull) { | ||
"" | ||
} else { | ||
toHiveString(row(ordinal), typ) | ||
} | ||
values.add(value) | ||
i += 1 | ||
} | ||
TColumn.stringVal(new TStringColumn(values, nulls)) | ||
} | ||
} | ||
|
||
override def toTColumnValue(ordinal: Int, row: Seq[_], types: Seq[Column]): TColumnValue = { | ||
getColumnType(types, ordinal).getRawType match { | ||
case BOOLEAN => toTTypeColumnVal(BOOLEAN_TYPE, row, ordinal) | ||
case TINYINT => toTTypeColumnVal(BINARY_TYPE, row, ordinal) | ||
case SMALLINT => toTTypeColumnVal(TINYINT_TYPE, row, ordinal) | ||
case INTEGER => toTTypeColumnVal(INT_TYPE, row, ordinal) | ||
case BIGINT => toTTypeColumnVal(BIGINT_TYPE, row, ordinal) | ||
case REAL => toTTypeColumnVal(FLOAT_TYPE, row, ordinal) | ||
case DOUBLE => toTTypeColumnVal(DOUBLE_TYPE, row, ordinal) | ||
case VARCHAR => toTTypeColumnVal(STRING_TYPE, row, ordinal) | ||
case _ => | ||
val tStrValue = new TStringValue | ||
if (row(ordinal) != null) { | ||
tStrValue.setValue( | ||
toHiveString(row(ordinal), types(ordinal).getTypeSignature)) | ||
} | ||
TColumnValue.stringVal(tStrValue) | ||
} | ||
} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
210 changes: 210 additions & 0 deletions
210
kyuubi-common/src/main/scala/org/apache/kyuubi/engine/schema/AbstractTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,210 @@ | ||
/* | ||
* 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.schema | ||
import java.nio.ByteBuffer | ||
import java.util.{ArrayList => JArrayList, BitSet => JBitSet, List => JList} | ||
|
||
import scala.collection.JavaConverters._ | ||
|
||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TTypeId._ | ||
import org.apache.kyuubi.util.RowSetUtils.bitSetToBuffer | ||
|
||
trait AbstractTRowSetGenerator[SchemaT, RowT, ColumnT] { | ||
|
||
protected def getColumnSizeFromSchemaType(schema: SchemaT): Int | ||
|
||
protected def getColumnType(schema: SchemaT, ordinal: Int): ColumnT | ||
|
||
protected def isColumnNullAt(row: RowT, ordinal: Int): Boolean | ||
|
||
protected def getColumnAs[T](row: RowT, ordinal: Int): T | ||
|
||
protected def toTColumn(rows: Seq[RowT], ordinal: Int, typ: ColumnT): TColumn | ||
|
||
protected def toTColumnValue(ordinal: Int, row: RowT, types: SchemaT): TColumnValue | ||
|
||
def toTRowSet( | ||
rows: Seq[RowT], | ||
schema: SchemaT, | ||
protocolVersion: TProtocolVersion): TRowSet = { | ||
if (protocolVersion.getValue < TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6.getValue) { | ||
toRowBasedSet(rows, schema) | ||
} else { | ||
toColumnBasedSet(rows, schema) | ||
} | ||
} | ||
|
||
def toRowBasedSet(rows: Seq[RowT], schema: SchemaT): TRowSet = { | ||
val rowSize = rows.length | ||
val tRows = new JArrayList[TRow](rowSize) | ||
var i = 0 | ||
while (i < rowSize) { | ||
val row = rows(i) | ||
var j = 0 | ||
val columnSize = getColumnSizeFromSchemaType(schema) | ||
val tColumnValues = new JArrayList[TColumnValue](columnSize) | ||
while (j < columnSize) { | ||
val columnValue = toTColumnValue(j, row, schema) | ||
tColumnValues.add(columnValue) | ||
j += 1 | ||
} | ||
i += 1 | ||
val tRow = new TRow(tColumnValues) | ||
tRows.add(tRow) | ||
} | ||
new TRowSet(0, tRows) | ||
} | ||
|
||
def toColumnBasedSet(rows: Seq[RowT], schema: SchemaT): TRowSet = { | ||
val rowSize = rows.length | ||
val tRowSet = new TRowSet(0, new JArrayList[TRow](rowSize)) | ||
var i = 0 | ||
val columnSize = getColumnSizeFromSchemaType(schema) | ||
val tColumns = new JArrayList[TColumn](columnSize) | ||
while (i < columnSize) { | ||
val tColumn = toTColumn(rows, i, getColumnType(schema, i)) | ||
tColumns.add(tColumn) | ||
i += 1 | ||
} | ||
tRowSet.setColumns(tColumns) | ||
tRowSet | ||
} | ||
|
||
protected def getOrSetAsNull[T]( | ||
rows: Seq[RowT], | ||
ordinal: Int, | ||
nulls: JBitSet, | ||
defaultVal: T): JList[T] = { | ||
val size = rows.length | ||
val ret = new JArrayList[T](size) | ||
var idx = 0 | ||
while (idx < size) { | ||
val row = rows(idx) | ||
val isNull = isColumnNullAt(row, ordinal) | ||
if (isNull) { | ||
nulls.set(idx, true) | ||
ret.add(defaultVal) | ||
} else { | ||
ret.add(getColumnAs[T](row, ordinal)) | ||
} | ||
idx += 1 | ||
} | ||
ret | ||
} | ||
|
||
protected def toTTypeColumnVal(typeId: TTypeId, row: RowT, ordinal: Int): TColumnValue = { | ||
def isNull = isColumnNullAt(row, ordinal) | ||
typeId match { | ||
case BOOLEAN_TYPE => | ||
val boolValue = new TBoolValue | ||
if (!isNull) boolValue.setValue(getColumnAs[java.lang.Boolean](row, ordinal)) | ||
TColumnValue.boolVal(boolValue) | ||
|
||
case BINARY_TYPE => | ||
val byteValue = new TByteValue | ||
if (!isNull) byteValue.setValue(getColumnAs[java.lang.Byte](row, ordinal)) | ||
TColumnValue.byteVal(byteValue) | ||
|
||
case TINYINT_TYPE => | ||
val tI16Value = new TI16Value | ||
if (!isNull) tI16Value.setValue(getColumnAs[java.lang.Short](row, ordinal)) | ||
TColumnValue.i16Val(tI16Value) | ||
|
||
case INT_TYPE => | ||
val tI32Value = new TI32Value | ||
if (!isNull) tI32Value.setValue(getColumnAs[java.lang.Integer](row, ordinal)) | ||
TColumnValue.i32Val(tI32Value) | ||
|
||
case BIGINT_TYPE => | ||
val tI64Value = new TI64Value | ||
if (!isNull) tI64Value.setValue(getColumnAs[java.lang.Long](row, ordinal)) | ||
TColumnValue.i64Val(tI64Value) | ||
|
||
case FLOAT_TYPE => | ||
val tDoubleValue = new TDoubleValue | ||
if (!isNull) tDoubleValue.setValue(getColumnAs[java.lang.Float](row, ordinal).toDouble) | ||
TColumnValue.doubleVal(tDoubleValue) | ||
|
||
case DOUBLE_TYPE => | ||
val tDoubleValue = new TDoubleValue | ||
if (!isNull) tDoubleValue.setValue(getColumnAs[java.lang.Double](row, ordinal)) | ||
TColumnValue.doubleVal(tDoubleValue) | ||
|
||
case STRING_TYPE => | ||
val tStringValue = new TStringValue | ||
if (!isNull) tStringValue.setValue(getColumnAs[String](row, ordinal)) | ||
TColumnValue.stringVal(tStringValue) | ||
|
||
case otherType => | ||
throw new UnsupportedOperationException(s"unsupported type $otherType for toTTypeColumnVal") | ||
} | ||
} | ||
|
||
protected def toTTypeColumn(typeId: TTypeId, rows: Seq[RowT], ordinal: Int): TColumn = { | ||
val nulls = new JBitSet() | ||
typeId match { | ||
case BOOLEAN_TYPE => | ||
val values = getOrSetAsNull[java.lang.Boolean](rows, ordinal, nulls, true) | ||
TColumn.boolVal(new TBoolColumn(values, nulls)) | ||
|
||
case BINARY_TYPE => | ||
val values = getOrSetAsNull[java.lang.Byte](rows, ordinal, nulls, 0.toByte) | ||
TColumn.byteVal(new TByteColumn(values, nulls)) | ||
|
||
case SMALLINT_TYPE => | ||
val values = getOrSetAsNull[java.lang.Short](rows, ordinal, nulls, 0.toShort) | ||
TColumn.i16Val(new TI16Column(values, nulls)) | ||
|
||
case TINYINT_TYPE => | ||
val values = getOrSetAsNull[java.lang.Short](rows, ordinal, nulls, 0.toShort) | ||
TColumn.i16Val(new TI16Column(values, nulls)) | ||
|
||
case INT_TYPE => | ||
val values = getOrSetAsNull[java.lang.Integer](rows, ordinal, nulls, 0) | ||
TColumn.i32Val(new TI32Column(values, nulls)) | ||
|
||
case BIGINT_TYPE => | ||
val values = getOrSetAsNull[java.lang.Long](rows, ordinal, nulls, 0L) | ||
TColumn.i64Val(new TI64Column(values, nulls)) | ||
|
||
case FLOAT_TYPE => | ||
val values = getOrSetAsNull[java.lang.Float](rows, ordinal, nulls, 0.toFloat) | ||
.asScala.map(n => java.lang.Double.valueOf(n.toString)).asJava | ||
TColumn.doubleVal(new TDoubleColumn(values, nulls)) | ||
|
||
case DOUBLE_TYPE => | ||
val values = getOrSetAsNull[java.lang.Double](rows, ordinal, nulls, 0.toDouble) | ||
TColumn.doubleVal(new TDoubleColumn(values, nulls)) | ||
|
||
case STRING_TYPE => | ||
val values = getOrSetAsNull[java.lang.String](rows, ordinal, nulls, "") | ||
TColumn.stringVal(new TStringColumn(values, nulls)) | ||
|
||
case ARRAY_TYPE => | ||
val values = getOrSetAsNull[Array[Byte]](rows, ordinal, nulls, Array()) | ||
.asScala | ||
.map(ByteBuffer.wrap) | ||
.asJava | ||
TColumn.binaryVal(new TBinaryColumn(values, nulls)) | ||
|
||
case otherType => | ||
throw new UnsupportedOperationException(s"unsupported type $otherType for toTTypeColumnVal") | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
209 changes: 0 additions & 209 deletions
209
kyuubi-server/src/main/scala/org/apache/kyuubi/sql/schema/RowSetHelper.scala
This file was deleted.
Oops, something went wrong.
78 changes: 78 additions & 0 deletions
78
kyuubi-server/src/main/scala/org/apache/kyuubi/sql/schema/ServerTRowSetGenerator.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,78 @@ | ||
/* | ||
* 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.sql.schema | ||
|
||
import org.apache.kyuubi.engine.schema.AbstractTRowSetGenerator | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift._ | ||
import org.apache.kyuubi.shaded.hive.service.rpc.thrift.TTypeId._ | ||
import org.apache.kyuubi.util.RowSetUtils._ | ||
|
||
class ServerTRowSetGenerator | ||
extends AbstractTRowSetGenerator[Schema, Row, TTypeId] { | ||
|
||
override def getColumnSizeFromSchemaType(schema: Schema): Int = schema.length | ||
|
||
override def getColumnType(schema: Schema, ordinal: Int): TTypeId = schema(ordinal).dataType | ||
|
||
override def isColumnNullAt(row: Row, ordinal: Int): Boolean = row.isNullAt(ordinal) | ||
|
||
override def getColumnAs[T](row: Row, ordinal: Int): T = row.getAs[T](ordinal) | ||
|
||
override def toTColumn(rows: Seq[Row], ordinal: Int, typ: TTypeId): TColumn = { | ||
val nulls = new java.util.BitSet() | ||
typ match { | ||
case t @ (BOOLEAN_TYPE | BINARY_TYPE | BINARY_TYPE | TINYINT_TYPE | INT_TYPE | | ||
BIGINT_TYPE | FLOAT_TYPE | DOUBLE_TYPE | STRING_TYPE) => | ||
toTTypeColumn(t, rows, ordinal) | ||
|
||
case _ => | ||
var i = 0 | ||
val rowSize = rows.length | ||
val values = new java.util.ArrayList[String](rowSize) | ||
while (i < rowSize) { | ||
val row = rows(i) | ||
val isNull = isColumnNullAt(row, ordinal) | ||
nulls.set(i, isNull) | ||
val value = if (isNull) { | ||
"" | ||
} else { | ||
(row.get(ordinal), typ).toString() | ||
} | ||
values.add(value) | ||
i += 1 | ||
} | ||
TColumn.stringVal(new TStringColumn(values, nulls)) | ||
} | ||
} | ||
|
||
override def toTColumnValue(ordinal: Int, row: Row, types: Schema): TColumnValue = { | ||
getColumnType(types, ordinal) match { | ||
case t @ (BOOLEAN_TYPE | BINARY_TYPE | BINARY_TYPE | TINYINT_TYPE | INT_TYPE | | ||
BIGINT_TYPE | FLOAT_TYPE | DOUBLE_TYPE | STRING_TYPE) => | ||
toTTypeColumnVal(t, row, ordinal) | ||
|
||
case _ => | ||
val tStrValue = new TStringValue | ||
if (!isColumnNullAt(row, ordinal)) { | ||
tStrValue.setValue((row.get(ordinal), types(ordinal).dataType).toString()) | ||
} | ||
TColumnValue.stringVal(tStrValue) | ||
} | ||
} | ||
|
||
} |