You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2021/08/02 21:09:46 UTC

[GitHub] [hudi] vinothchandar commented on a change in pull request #3277: [HUDI-2182] Support Compaction Command For Spark Sql

vinothchandar commented on a change in pull request #3277:
URL: https://github.com/apache/hudi/pull/3277#discussion_r681271066



##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.hudi.command
+
+import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils}
+import org.apache.hudi.client.WriteStatus
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.exception.HoodieException
+import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.types.StringType
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+case class CompactionHoodiePathCommand(path: String,
+  operation: CompactionOperation, instantTimestamp: Option[Long] = None)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
+      .setConf(sparkSession.sessionState.newHadoopConf()).build()
+
+    assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ,
+      s"Must compaction on a Merge On Read table.")
+    val schemaUtil = new TableSchemaResolver(metaClient)
+    val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
+
+    val parameters = HoodieWriterUtils.parametersWithWriteDefaults(
+        HoodieSqlUtils.withSparkConf(sparkSession, Map.empty)(
+          Map(
+            DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key() -> HoodieTableType.MERGE_ON_READ.name()
+          )
+        )
+      )
+    val jsc = new JavaSparkContext(sparkSession.sparkContext)
+    val client = DataSourceUtils.createHoodieClient(jsc, schemaStr, path,
+      metaClient.getTableConfig.getTableName, parameters)
+
+    operation match {
+      case SCHEDULE =>

Review comment:
       can we just reuse the `CompactionAdminClient` or other tools we use to run compactions. there are classes in org.apache.hudi.utilities.*  for cleaning, compaction, clustering already. more we reuse the better

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.hudi.command
+
+import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils}
+import org.apache.hudi.client.WriteStatus
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.exception.HoodieException
+import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.types.StringType
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+case class CompactionHoodiePathCommand(path: String,
+  operation: CompactionOperation, instantTimestamp: Option[Long] = None)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
+      .setConf(sparkSession.sessionState.newHadoopConf()).build()
+
+    assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ,
+      s"Must compaction on a Merge On Read table.")
+    val schemaUtil = new TableSchemaResolver(metaClient)
+    val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
+
+    val parameters = HoodieWriterUtils.parametersWithWriteDefaults(
+        HoodieSqlUtils.withSparkConf(sparkSession, Map.empty)(
+          Map(
+            DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key() -> HoodieTableType.MERGE_ON_READ.name()
+          )
+        )
+      )
+    val jsc = new JavaSparkContext(sparkSession.sparkContext)
+    val client = DataSourceUtils.createHoodieClient(jsc, schemaStr, path,
+      metaClient.getTableConfig.getTableName, parameters)
+
+    operation match {
+      case SCHEDULE =>

Review comment:
       the advantage is all the error checking etc should be pushed into those classes once and we keep the layer here very thin

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionHoodiePathCommand.scala
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.hudi.command
+
+import org.apache.hudi.{DataSourceUtils, DataSourceWriteOptions, HoodieWriterUtils}
+import org.apache.hudi.client.WriteStatus
+import org.apache.hudi.common.model.HoodieTableType
+import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieTimeline}
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
+import org.apache.hudi.common.util.{Option => HOption}
+import org.apache.hudi.exception.HoodieException
+import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.{CompactionOperation, RUN, SCHEDULE}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.hudi.HoodieSqlUtils
+import org.apache.spark.sql.types.StringType
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+case class CompactionHoodiePathCommand(path: String,
+  operation: CompactionOperation, instantTimestamp: Option[Long] = None)
+  extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
+      .setConf(sparkSession.sessionState.newHadoopConf()).build()
+
+    assert(metaClient.getTableType == HoodieTableType.MERGE_ON_READ,
+      s"Must compaction on a Merge On Read table.")
+    val schemaUtil = new TableSchemaResolver(metaClient)
+    val schemaStr = schemaUtil.getTableAvroSchemaWithoutMetadataFields.toString
+
+    val parameters = HoodieWriterUtils.parametersWithWriteDefaults(
+        HoodieSqlUtils.withSparkConf(sparkSession, Map.empty)(
+          Map(
+            DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key() -> HoodieTableType.MERGE_ON_READ.name()
+          )
+        )
+      )
+    val jsc = new JavaSparkContext(sparkSession.sparkContext)
+    val client = DataSourceUtils.createHoodieClient(jsc, schemaStr, path,
+      metaClient.getTableConfig.getTableName, parameters)
+
+    operation match {
+      case SCHEDULE =>
+        val instantTime = instantTimestamp.map(_.toString).getOrElse(HoodieActiveTimeline.createNewInstantTime)
+        if (client.scheduleCompactionAtInstant(instantTime, HOption.empty[java.util.Map[String, String]])) {
+          Seq(Row(instantTime))
+        } else {
+          Seq(Row(null))
+        }
+      case RUN =>
+        // Do compaction
+        val timeLine = metaClient.getActiveTimeline
+         val pendingCompactionInstants = timeLine.getWriteTimeline.getInstants.iterator().asScala
+          .filter(p => p.getAction == HoodieTimeline.COMPACTION_ACTION)
+           .map(_.getTimestamp)
+          .toSeq.sortBy(f => f)
+        val willCompactionInstants = if (instantTimestamp.isEmpty) {
+           if (pendingCompactionInstants.nonEmpty) {
+             pendingCompactionInstants
+           } else { // If there are no pending compaction, schedule to generate one.
+             // CompactionHoodiePathCommand will return instanceTime for SCHEDULE.
+             Seq(CompactionHoodiePathCommand(path, CompactionOperation.SCHEDULE)
+               .run(sparkSession).take(1).get(0).getString(0)).filter(_ != null)
+           }
+        } else {
+          // Check if the compaction timestamp has exists in the pending compaction
+          if (pendingCompactionInstants.contains(instantTimestamp.get.toString)) {
+            Seq(instantTimestamp.get.toString)
+          } else {
+            throw new IllegalArgumentException(s"Compaction instant: ${instantTimestamp.get} is not found in $path," +
+              s" Available pending compaction instants are: ${pendingCompactionInstants.mkString(",")} ")
+          }
+        }
+        if (willCompactionInstants.isEmpty) {
+          logInfo(s"No need to compaction on $path")
+          Seq.empty[Row]
+        } else {
+          logInfo(s"Run compaction at instants: [${willCompactionInstants.mkString(",")}] on $path")
+          val startTs = System.currentTimeMillis()

Review comment:
       HoodieTimer for timing?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlCommon.g4
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.
+ */
+grammar HoodieSqlCommon;
+
+ singleStatement
+    : statement EOF
+    ;
+
+statement
+    : compactionStatement                                              #compactionCommand
+    | .*?                                                              #passThrough
+    ;
+
+ compactionStatement
+    : operation = (RUN | SCHEDULE) COMPACTION  ON tableIdentifier (AT timestamp = NUMBER)?    #compactionOnTable

Review comment:
       instead of `timestamp`, can we use Hudi terms. ` (AT instanttime = NUMBER) `

##########
File path: hudi-spark-datasource/hudi-spark/pom.xml
##########
@@ -157,6 +157,23 @@
         <groupId>org.jacoco</groupId>
         <artifactId>jacoco-maven-plugin</artifactId>
       </plugin>
+      <plugin>
+        <groupId>org.antlr</groupId>
+        <artifactId>antlr4-maven-plugin</artifactId>
+        <version>4.7</version>

Review comment:
       pull version into a property?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlCommon.g4
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.
+ */
+grammar HoodieSqlCommon;
+
+ singleStatement
+    : statement EOF
+    ;
+
+statement
+    : compactionStatement                                              #compactionCommand
+    | .*?                                                              #passThrough
+    ;
+
+ compactionStatement
+    : operation = (RUN | SCHEDULE) COMPACTION  ON tableIdentifier (AT timestamp = NUMBER)?    #compactionOnTable
+    | operation = (RUN | SCHEDULE) COMPACTION  ON path = STRING   (AT timestamp = NUMBER)?    #compactionOnPath

Review comment:
       In this model, how does the user configure compaction strategy and its input configs?

##########
File path: hudi-spark-datasource/hudi-spark/src/main/antlr4/org/apache/hudi/spark/sql/parser/HoodieSqlCommon.g4
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.
+ */
+grammar HoodieSqlCommon;
+
+ singleStatement
+    : statement EOF
+    ;
+
+statement
+    : compactionStatement                                              #compactionCommand
+    | .*?                                                              #passThrough
+    ;
+
+ compactionStatement
+    : operation = (RUN | SCHEDULE) COMPACTION  ON tableIdentifier (AT timestamp = NUMBER)?    #compactionOnTable
+    | operation = (RUN | SCHEDULE) COMPACTION  ON path = STRING   (AT timestamp = NUMBER)?    #compactionOnPath

Review comment:
       Was doing some research on paths we can take here. 
   - Instead of extending SQL. Is it better to just surface this via stored procedures. `EXEC` statements? We can accept parameters easily? 
   - I do see folks like snowflake using `ALTER table`  for clustering, because clustering keys are setup during table creation. But we probably want better flexibility. 
   https://docs.snowflake.com/en/user-guide/tables-auto-reclustering.html#suspending-automatic-clustering-for-a-table
   

##########
File path: hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CompactionShowHoodieTableCommand.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.hudi.command
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.hudi.HoodieSqlUtils.getTableLocation
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+case class CompactionShowHoodieTableCommand(table: CatalogTable, limit: Int)

Review comment:
       we can drop `Hoodie` from the class name?




-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org