ulysses-you commented on a change in pull request #990: URL: https://github.com/apache/incubator-kyuubi/pull/990#discussion_r698272949
########## File path: dev/kyuubi-extension-spark-3-1/src/main/scala/org/apache/kyuubi/sql/zorder/OptimizeZorderCommand.scala ########## @@ -0,0 +1,40 @@ +/* + * 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.zorder + +import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, UnaryNode} + +case class OptimizeZorderCommand(child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output +} + +object OptimizeZorderCommand { + def apply(tableIdent: Seq[String], + whereExp: Option[Expression], + sortOrder: Seq[SortOrder]): OptimizeZorderCommand = { + val table = UnresolvedRelation(tableIdent) + val child = whereExp match { + case Some(x) => Project(Seq(UnresolvedStar(None)), Filter(x, table)) + case None => Project(Seq(UnresolvedStar(None)), table) Review comment: Do we really need `Project` ? ########## File path: dev/kyuubi-extension-spark-3-1/src/main/scala/org/apache/spark/sql/execution/ZorderOrdering.scala ########## @@ -0,0 +1,100 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BaseOrdering, SortOrder} +import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} + +class ZorderOrdering(ordering: Seq[SortOrder], + schema: StructType) extends BaseOrdering { + private lazy val projectList = ordering.map { + case o if o.child.isInstanceOf[AttributeReference] => + o.child.asInstanceOf[AttributeReference] + } + + private lazy val indexFields = { + val columnsMap = this.schema.fields.map(item => (item.name, item)).toMap + projectList.map(item => { + val newItem = columnsMap(item.name) + (this.schema.fields.indexOf(newItem), newItem) + }) + } + + private def getZvalue(row: InternalRow): Array[Byte] = { + val values = indexFields.map { + case (index, field) => + field.dataType match { + case BooleanType => + ZorderBytesUtils.booleanToByte( + if (row.isNullAt(index)) false else row.getBoolean(index) + ) + case ByteType => + ZorderBytesUtils.byteToByte( + if (row.isNullAt(index)) Byte.MaxValue else row.getByte(index) + ) + case ShortType => + ZorderBytesUtils.shortToByte( + if (row.isNullAt(index)) Short.MaxValue else row.getShort(index) + ) + case IntegerType => + ZorderBytesUtils.intToByte( + if (row.isNullAt(index)) Int.MaxValue else row.getInt(index) + ) + case LongType => + ZorderBytesUtils.longToByte( + if (row.isNullAt(index)) Long.MaxValue else row.getLong(index) + ) + case FloatType => + ZorderBytesUtils.floatToByte( + if (row.isNullAt(index)) Float.MaxValue else row.getFloat(index) + ) + case DoubleType => + ZorderBytesUtils.doubleToByte( + if (row.isNullAt(index)) Double.MaxValue else row.getDouble(index) + ) + case StringType => + ZorderBytesUtils.stringToByte( + if (row.isNullAt(index)) "" else row.getString(index) + ) + case _ => null +// case DateType +// case TimestampType +// case DecimalType Review comment: if we don't support them now, we can add exception during analysis ########## File path: dev/kyuubi-extension-spark-3-1/src/main/scala/org/apache/spark/sql/execution/ZorderOrdering.scala ########## @@ -0,0 +1,100 @@ +/* + * 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.spark.sql.execution + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BaseOrdering, SortOrder} +import org.apache.spark.sql.types.{BinaryType, BooleanType, ByteType, DoubleType, FloatType, IntegerType, LongType, ShortType, StringType, StructType} + +class ZorderOrdering(ordering: Seq[SortOrder], + schema: StructType) extends BaseOrdering { Review comment: Can we make a new `Expression` which return the binary type value and produced by Z-order ? IMO we can apply Z-order with Spark `Ordering` simply, like `Sort(zorder(c1, c2, c3), ...)` instead. We can still name it as `zorder by c1, c2, c3` in `.g4`. The benifits of this approach is: * more cleaner since less code copy * support global z-order, otherwise we need to add a new partitioner -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
