You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2021/09/02 23:35:00 UTC

[jira] [Commented] (HUDI-2101) support z-order for hudi

    [ https://issues.apache.org/jira/browse/HUDI-2101?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17409155#comment-17409155 ] 

ASF GitHub Bot commented on HUDI-2101:
--------------------------------------

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



##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
##########
@@ -241,6 +241,27 @@ private synchronized FileSystemViewManager getViewManager() {
    */
   public abstract HoodieWriteMetadata<O> insertOverwriteTable(HoodieEngineContext context, String instantTime, I records);
 
+  /**
+   * Replaces all the existing records of the Hoodie table and optimize data layout,
+   * for the partition paths contained in input records.
+   *
+   * @param context HoodieEngineContext
+   * @param instantTime Instant time for the replace action
+   * @param records input records
+   * @return HoodieWriteMetadata
+   */
+  public abstract HoodieWriteMetadata<O> optimize(HoodieEngineContext context, String instantTime, I records);
+
+  /**
+   * update statistics info for current table.
+   * now only support OPTIMIZE operation, to do support other operation type.
+   *
+   * @param context HoodieEngineContext
+   * @param instantTime Instant time for the replace action
+   * @param isOptimizeOperation whether current operation is OPTIMIZE type
+   */
+  public abstract void updateStatistics(HoodieEngineContext context, List<HoodieWriteStat> stats, String instantTime, Boolean isOptimizeOperation);

Review comment:
       In the future this should go into the `HoodieTableMetadata` API IMO. I know we don't have the full RFC-27 in place yet. But something to streamline before we release.

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
##########
@@ -149,6 +155,37 @@ public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String i
     return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute();
   }
 
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> optimize(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {

Review comment:
       I wonder why z-ordering cannot be implemented as a just another clustering strategy? 

##########
File path: hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
##########
@@ -76,6 +76,7 @@
   private static final long serialVersionUID = 1L;
   private static final Logger LOG = LogManager.getLogger(HoodieTableMetaClient.class);
   public static final String METAFOLDER_NAME = ".hoodie";
+  public static final String ZINDEX_NAME = ".index";

Review comment:
       nit.`.zindex`

##########
File path: hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/Zoptimize.scala
##########
@@ -0,0 +1,826 @@
+/*
+ * 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
+
+import java.sql.Date
+import java.util.concurrent.{Executors, ThreadPoolExecutor}
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileStatus, Path}
+import org.apache.hudi.config.HoodieOptimizeConfig
+import org.apache.parquet.hadoop.ParquetFileReader
+import org.apache.spark.SparkContext
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.expressions.{Alias, And, Ascending, Attribute, AttributeReference, BoundReference, EqualNullSafe, EqualTo, Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, SortOrder, StartsWith, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering
+import org.apache.spark.sql.execution.datasources.PartitionedFile
+import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
+import org.apache.spark.sql.functions._
+import org.apache.hudi.optimize.ZOrderingUtil
+import org.apache.spark.sql.hudi.execution._
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.vectorized.ColumnarBatch
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.{MutablePair, SerializableConfiguration}
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import scala.concurrent.duration._
+import scala.concurrent.{ExecutionContext, Future}
+
+object Zoptimize {

Review comment:
       hmmm. can this be in Java? So we can avoid scala in the hudi-spark-client module?

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/optimize/UnsafeAccess.java
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.hudi.optimize;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.misc.Unsafe;
+
+import java.lang.reflect.Field;
+import java.nio.ByteOrder;
+import java.security.AccessController;
+import java.security.PrivilegedAction;
+
+public class UnsafeAccess {

Review comment:
       +1 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
##########
@@ -149,6 +155,37 @@ public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String i
     return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute();
   }
 
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> optimize(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkOptimizeWriteCommitActionExecutor((HoodieSparkEngineContext)context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public void updateStatistics(HoodieEngineContext context, List<HoodieWriteStat> stats, String instantTime, Boolean isOptimizeOperation) {
+    // deal with z-order/hilbert statistic info
+    if (isOptimizeOperation) {
+      updateOptimizeOperationStatistics(context, stats, instantTime);
+    }
+  }
+
+  private void updateOptimizeOperationStatistics(HoodieEngineContext context, List<HoodieWriteStat> stats, String instantTime) {
+    String cols = config.getOptimizeSortColumns();
+    String saveMode = config.getOptimizeStatisticsSaveMode();
+    String basePath = metaClient.getBasePath();
+    String indexPath = metaClient.getZindexPath();
+    List<String> validateCommits = metaClient.getCommitsTimeline()
+        .filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList());
+    List<String> touchFiles = stats.stream().map(s -> new Path(basePath, s.getPath()).toString()).collect(Collectors.toList());
+    if (touchFiles.isEmpty() || cols.isEmpty() || indexPath.isEmpty()) {
+      LOG.warn("save nothing to index table");
+      return;
+    }
+    HoodieSparkEngineContext sparkEngineContext = (HoodieSparkEngineContext)context;
+    Zoptimize$.MODULE$.saveStatisticsInfo(sparkEngineContext

Review comment:
       So this is getting stored by Spark internally? can you please clarify how this is maintained? do we remove stats due to cleaning i.e files that are no longer part of the table? 

##########
File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java
##########
@@ -149,6 +155,37 @@ public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String i
     return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute();
   }
 
+  @Override
+  public HoodieWriteMetadata<JavaRDD<WriteStatus>> optimize(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
+    return new SparkOptimizeWriteCommitActionExecutor((HoodieSparkEngineContext)context, config, this, instantTime, records).execute();
+  }
+
+  @Override
+  public void updateStatistics(HoodieEngineContext context, List<HoodieWriteStat> stats, String instantTime, Boolean isOptimizeOperation) {
+    // deal with z-order/hilbert statistic info
+    if (isOptimizeOperation) {
+      updateOptimizeOperationStatistics(context, stats, instantTime);
+    }
+  }
+
+  private void updateOptimizeOperationStatistics(HoodieEngineContext context, List<HoodieWriteStat> stats, String instantTime) {
+    String cols = config.getOptimizeSortColumns();
+    String saveMode = config.getOptimizeStatisticsSaveMode();
+    String basePath = metaClient.getBasePath();
+    String indexPath = metaClient.getZindexPath();
+    List<String> validateCommits = metaClient.getCommitsTimeline()
+        .filterCompletedInstants().getInstants().map(f -> f.getTimestamp()).collect(Collectors.toList());
+    List<String> touchFiles = stats.stream().map(s -> new Path(basePath, s.getPath()).toString()).collect(Collectors.toList());
+    if (touchFiles.isEmpty() || cols.isEmpty() || indexPath.isEmpty()) {
+      LOG.warn("save nothing to index table");
+      return;
+    }
+    HoodieSparkEngineContext sparkEngineContext = (HoodieSparkEngineContext)context;
+    Zoptimize$.MODULE$.saveStatisticsInfo(sparkEngineContext

Review comment:
       Looks like its getting stored as internal Spark tables




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


> support z-order for hudi
> ------------------------
>
>                 Key: HUDI-2101
>                 URL: https://issues.apache.org/jira/browse/HUDI-2101
>             Project: Apache Hudi
>          Issue Type: Sub-task
>          Components: Spark Integration
>            Reporter: tao meng
>            Assignee: tao meng
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 0.10.0
>
>
> support z-order for hudi to optimze the query



--
This message was sent by Atlassian Jira
(v8.3.4#803005)