ShreyeshArangath commented on code in PR #1995:
URL: https://github.com/apache/auron/pull/1995#discussion_r2789534848


##########
spark-extension-shims-spark/src/main/scala/org/apache/spark/sql/hive/execution/auron/plan/HiveConvertProvider.scala:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.hive.execution.auron.plan
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.auron.AuronConverters.getBooleanConf
+import org.apache.spark.sql.auron.{AuronConvertProvider, AuronConverters}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.hive.execution.HiveTableScanExec
+
+class HiveConvertProvider extends AuronConvertProvider with Logging {
+  override def isEnabled: Boolean =
+    getBooleanConf("spark.auron.enable.hiveTable", defaultValue = true)
+
+  def enableHiveTableScanExec: Boolean =
+    getBooleanConf("spark.auron.enable.hiveTableScanExec", defaultValue = 
false)

Review Comment:
   Should we be adding these to @SparkAuronConfiguration.java? 



##########
spark-extension-shims-spark/src/main/scala/org/apache/spark/sql/hive/execution/auron/plan/NativeHIveTableScanExec.scala:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.hive.execution.auron.plan
+
+import org.apache.auron.metric.SparkMetricNode
+import org.apache.auron.{protobuf => pb}
+import org.apache.hadoop.conf.Configurable
+import org.apache.hadoop.hive.ql.exec.Utilities
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat
+import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
+import org.apache.hadoop.hive.ql.metadata.{Table => HiveTable}
+import org.apache.hadoop.hive.ql.plan.TableDesc
+import org.apache.hadoop.hive.serde.serdeConstants
+import 
org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption
+import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorUtils, 
StructObjectInspector}
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils
+import org.apache.hadoop.io.Writable
+import org.apache.hadoop.mapred.{FileSplit, InputFormat, JobConf}
+import org.apache.hadoop.mapreduce.{InputFormat => newInputClass}
+import org.apache.hadoop.util.ReflectionUtils
+import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.HADOOP_RDD_IGNORE_EMPTY_SPLITS
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.auron.{NativeRDD, Shims}
+import org.apache.spark.sql.catalyst.expressions.{AttributeMap, 
GenericInternalRow}
+import org.apache.spark.sql.execution.datasources.{FilePartition, 
PartitionedFile}
+import org.apache.spark.sql.hive.client.HiveClientImpl
+import org.apache.spark.sql.hive.execution.HiveTableScanExec
+import org.apache.spark.sql.hive.{HadoopTableReader, HiveShim}
+import org.apache.spark.{Partition, TaskContext}
+
+import java.util.UUID
+import scala.collection.JavaConverters._
+import scala.collection.mutable.ArrayBuffer
+
+case class NativeHiveTableScanExec(basedHiveScan: HiveTableScanExec)
+  extends NativeHiveTableScanBase(basedHiveScan)
+    with Logging {
+
+  @transient private lazy val nativeTable: HiveTable = 
HiveClientImpl.toHiveTable(relation.tableMeta)
+  @transient private lazy val fileFormat = 
HiveTableUtil.getFileFormat(nativeTable.getInputFormatClass)
+  @transient private lazy val nativeTableDesc = new TableDesc(
+    nativeTable.getInputFormatClass,
+    nativeTable.getOutputFormatClass,
+    nativeTable.getMetadata)
+
+  @transient private lazy val nativeHadoopConf = {
+    val hiveConf = 
SparkSession.getActiveSession.get.sessionState.newHadoopConf()
+    // append columns ids and names before broadcast
+    val columnOrdinals = AttributeMap(relation.dataCols.zipWithIndex)
+    val neededColumnIDs = output.flatMap(columnOrdinals.get).map(o => o: 
Integer)
+    val neededColumnNames = output.filter(columnOrdinals.contains).map(_.name)
+
+    HiveShim.appendReadColumns(hiveConf, neededColumnIDs, neededColumnNames)
+
+    val deserializer = 
nativeTableDesc.getDeserializerClass.getConstructor().newInstance()
+    deserializer.initialize(hiveConf, nativeTableDesc.getProperties)
+
+    // Specifies types and object inspectors of columns to be scanned.
+    val structOI = ObjectInspectorUtils
+      .getStandardObjectInspector(
+        deserializer.getObjectInspector,
+        ObjectInspectorCopyOption.JAVA)
+      .asInstanceOf[StructObjectInspector]
+
+    val columnTypeNames = structOI
+      .getAllStructFieldRefs.asScala
+      .map(_.getFieldObjectInspector)
+      .map(TypeInfoUtils.getTypeInfoFromObjectInspector(_).getTypeName)
+      .mkString(",")
+
+    hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames)
+    hiveConf.set(serdeConstants.LIST_COLUMNS, 
relation.dataCols.map(_.name).mkString(","))
+    hiveConf
+  }
+
+  private val minPartitions = if 
(SparkSession.getActiveSession.get.sparkContext.isLocal) {
+    0 // will splitted based on block by default.
+  } else {
+    math.max(nativeHadoopConf.getInt("mapreduce.job.maps", 1),
+      SparkSession.getActiveSession.get.sparkContext.defaultMinPartitions)
+  }
+
+  private val ignoreEmptySplits =
+    
SparkSession.getActiveSession.get.sparkContext.conf.get(HADOOP_RDD_IGNORE_EMPTY_SPLITS)
+
+  override val nodeName: String =
+    s"NativeHiveTableScan $tableName"
+
+  override def doExecuteNative(): NativeRDD = {
+    val nativeMetrics = SparkMetricNode(
+      metrics,
+      Nil,
+      Some({
+        case ("bytes_scanned", v) =>
+          val inputMetric = TaskContext.get.taskMetrics().inputMetrics
+          inputMetric.incBytesRead(v)
+        case ("output_rows", v) =>
+          val inputMetric = TaskContext.get.taskMetrics().inputMetrics
+          inputMetric.incRecordsRead(v)
+        case _ =>
+      }))
+    val nativeFileSchema = this.nativeFileSchema
+    val nativeFileGroups = this.nativeFileGroups
+    val nativePartitionSchema = this.nativePartitionSchema
+
+    val projection = schema.map(field => 
relation.schema.fieldIndex(field.name))
+    val broadcastedHadoopConf = this.broadcastedHadoopConf
+    val numPartitions = partitions.length
+
+    new NativeRDD(
+      sparkContext,
+      nativeMetrics,
+      partitions.asInstanceOf[Array[Partition]],
+      None,
+      Nil,
+      rddShuffleReadFull = true,
+      (partition, _) => {
+        val resourceId = s"NativeHiveTableScan:${UUID.randomUUID().toString}"
+        putJniBridgeResource(resourceId, broadcastedHadoopConf)
+
+        val nativeFileGroup = 
nativeFileGroups(partition.asInstanceOf[FilePartition])
+        val nativeFileScanConf = pb.FileScanExecConf
+          .newBuilder()
+          .setNumPartitions(numPartitions)
+          .setPartitionIndex(partition.index)
+          .setStatistics(pb.Statistics.getDefaultInstance)
+          .setSchema(nativeFileSchema)
+          .setFileGroup(nativeFileGroup)
+          .addAllProjection(projection.map(Integer.valueOf).asJava)
+          .setPartitionSchema(nativePartitionSchema)
+          .build()
+        fileFormat match {
+          case "orc" =>
+            val nativeOrcScanExecBuilder = pb.OrcScanExecNode
+              .newBuilder()
+              .setBaseConf(nativeFileScanConf)
+              .setFsResourceId(resourceId)
+              .addAllPruningPredicates(new java.util.ArrayList()) // not 
support this filter

Review Comment:
   So we will do a full table scan here? If that's a case, do you mind creating 
an issue and linking it here? 



##########
spark-extension-shims-spark/src/main/scala/org/apache/spark/sql/hive/execution/auron/plan/HiveConvertProvider.scala:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.hive.execution.auron.plan
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.auron.AuronConverters.getBooleanConf
+import org.apache.spark.sql.auron.{AuronConvertProvider, AuronConverters}
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.hive.execution.HiveTableScanExec
+
+class HiveConvertProvider extends AuronConvertProvider with Logging {
+  override def isEnabled: Boolean =
+    getBooleanConf("spark.auron.enable.hiveTable", defaultValue = true)
+
+  def enableHiveTableScanExec: Boolean =
+    getBooleanConf("spark.auron.enable.hiveTableScanExec", defaultValue = 
false)
+
+  override def isSupported(exec: SparkPlan): Boolean =
+    exec match {
+      case e: HiveTableScanExec if enableHiveTableScanExec &&
+        e.relation.tableMeta.provider.isDefined &&
+        e.relation.tableMeta.provider.get.equals("hive") =>
+        true
+    case _ => false
+  }

Review Comment:
   Slightly hard to distinguish when to use `isSupported` v/s 
`enableHiveTableScanExec` flag. Do you mind adding documentation here? 



-- 
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]

Reply via email to