[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-08-01 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieOptimizeConfig.java
##
@@ -0,0 +1,133 @@
+/*
+ * 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.config;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Hoodie Configs for Data layout optimize.
+ */
+public class HoodieOptimizeConfig extends HoodieConfig {
+  // Any Data layout optimize params can be saved with this prefix
+  public static final String DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX = 
"hoodie.data.layout.optimize.";
+  public static final ConfigProperty DATA_LAYOUT_STRATEGY = ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "strategy")
+  .defaultValue("z-order")
+  .sinceVersion("0.10.0")
+  .withDocumentation("config to provide a way to optimize data layout for 
table, current only support z-order and hilbert");
+
+  public static final ConfigProperty DATA_LAYOUT_BUILD_CURVE_METHOD = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "build.curve.optimize.method")
+  .defaultValue("directly")
+  .sinceVersion("0.10.0")
+  .withDocumentation("Config to provide whether use directly/sample method 
to build curve optimize for data layout,"
+  + " build curve_optimize by directly method is faster than by sample 
method, however sample method produce a better data layout");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SAMPLE_NUMBER 
= ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sample.number")
+  .defaultValue("20")
+  .sinceVersion("0.10.0")
+  .withDocumentation("when set" + DATA_LAYOUT_BUILD_CURVE_METHOD.key() + " 
to sample method, sample number need to be set for it."
+  + " larger number means better layout result, but more memory 
consumer");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SORT_COLUMNS = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sort.columns")
+  .defaultValue("")
+  .sinceVersion("0.10.0")
+  .withDocumentation("sort columns for build curve optimize. default value 
is empty string which means no sort."
+  + " more sort columns you specify, the worse data layout result. No 
more than 4 are recommended");
+
+  public static final ConfigProperty DATA_LAYOUT_DATA_SKIPPING_ENABLE = 
ConfigProperty

Review comment:
   > sorry, i cannot get the point. may be i miss somethings
   
   `ConfigProperty`




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-08-01 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieOptimizeConfig.java
##
@@ -0,0 +1,133 @@
+/*
+ * 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.config;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Hoodie Configs for Data layout optimize.
+ */
+public class HoodieOptimizeConfig extends HoodieConfig {
+  // Any Data layout optimize params can be saved with this prefix
+  public static final String DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX = 
"hoodie.data.layout.optimize.";
+  public static final ConfigProperty DATA_LAYOUT_STRATEGY = ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "strategy")
+  .defaultValue("z-order")
+  .sinceVersion("0.10.0")
+  .withDocumentation("config to provide a way to optimize data layout for 
table, current only support z-order and hilbert");
+
+  public static final ConfigProperty DATA_LAYOUT_BUILD_CURVE_METHOD = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "build.curve.optimize.method")
+  .defaultValue("directly")
+  .sinceVersion("0.10.0")
+  .withDocumentation("Config to provide whether use directly/sample method 
to build curve optimize for data layout,"
+  + " build curve_optimize by directly method is faster than by sample 
method, however sample method produce a better data layout");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SAMPLE_NUMBER 
= ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sample.number")
+  .defaultValue("20")
+  .sinceVersion("0.10.0")
+  .withDocumentation("when set" + DATA_LAYOUT_BUILD_CURVE_METHOD.key() + " 
to sample method, sample number need to be set for it."
+  + " larger number means better layout result, but more memory 
consumer");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SORT_COLUMNS = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sort.columns")
+  .defaultValue("")
+  .sinceVersion("0.10.0")
+  .withDocumentation("sort columns for build curve optimize. default value 
is empty string which means no sort."
+  + " more sort columns you specify, the worse data layout result. No 
more than 4 are recommended");
+
+  public static final ConfigProperty DATA_LAYOUT_DATA_SKIPPING_ENABLE = 
ConfigProperty

Review comment:
   > sorry, i cannot get the point. may be i miss somethings
   
   ConfigProperty




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-30 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/OptimizeTableByCurve.scala
##
@@ -0,0 +1,83 @@
+/*
+ * 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 org.apache.hudi.config.HoodieOptimizeConfig
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.{And, Expression, Literal, 
PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, 
Project}
+import org.apache.spark.sql.execution.command.RunnableCommand
+
+abstract class OptimizeTableByCurve(
+target: LogicalPlan,
+condition: Option[Expression],
+orderFields: Seq[Expression],
+outputFileNum: Option[Int],
+options: Option[Map[String, String]],
+curveName: String = "z-order") extends RunnableCommand with Logging with 
PredicateHelper {
+
+  val z_orderFields = 
orderFields.map(Zoptimize.getTargetColNameParts(_).mkString("."))
+
+  val optimizeCondition = condition.getOrElse(Literal.TrueLiteral)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val optimizeDF = curveName match {
+  case "z-order" =>
+buildZData(sparkSession)
+  // to do support hibert curve
+  case "hilbert" =>
+throw new SparkException(s"now not support hilbert optimize")
+  case other =>
+throw new SparkException(s"only support z-order curve but find: 
${other}")

Review comment:
   fix the description.




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-30 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
##
@@ -460,6 +487,17 @@ protected void preCommit(String instantTime, 
HoodieCommitMetadata metadata) {
 Option.of(metadata), config, 
txnManager.getLastCompletedTransactionOwner());
   }
 
+  @Override
+  protected void saveStatisticsInfo(List touchFiles, String cols, 
String indexPath, String saveMode) {
+if (touchFiles.isEmpty() || cols.isEmpty() || indexPath.isEmpty()) {
+  LOG.warn("save nothing to index table");

Review comment:
   save statistics into .hoodie/.index/xxx ?

##
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java
##
@@ -460,6 +487,17 @@ protected void preCommit(String instantTime, 
HoodieCommitMetadata metadata) {
 Option.of(metadata), config, 
txnManager.getLastCompletedTransactionOwner());
   }
 
+  @Override
+  protected void saveStatisticsInfo(List touchFiles, String cols, 
String indexPath, String saveMode) {
+if (touchFiles.isEmpty() || cols.isEmpty() || indexPath.isEmpty()) {
+  LOG.warn("save nothing to index table");

Review comment:
   save statistics into .hoodie/index/xxx ?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-30 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieOptimizeConfig.java
##
@@ -0,0 +1,133 @@
+/*
+ * 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.config;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Hoodie Configs for Data layout optimize.
+ */
+public class HoodieOptimizeConfig extends HoodieConfig {
+  // Any Data layout optimize params can be saved with this prefix
+  public static final String DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX = 
"hoodie.data.layout.optimize.";
+  public static final ConfigProperty DATA_LAYOUT_STRATEGY = ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "strategy")
+  .defaultValue("z-order")
+  .sinceVersion("0.10.0")
+  .withDocumentation("config to provide a way to optimize data layout for 
table, current only support z-order and hilbert");
+
+  public static final ConfigProperty DATA_LAYOUT_BUILD_CURVE_METHOD = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "build.curve.optimize.method")
+  .defaultValue("directly")
+  .sinceVersion("0.10.0")
+  .withDocumentation("Config to provide whether use directly/sample method 
to build curve optimize for data layout,"
+  + " build curve_optimize by directly method is faster than by sample 
method, however sample method produce a better data layout");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SAMPLE_NUMBER 
= ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sample.number")
+  .defaultValue("20")
+  .sinceVersion("0.10.0")
+  .withDocumentation("when set" + DATA_LAYOUT_BUILD_CURVE_METHOD.key() + " 
to sample method, sample number need to be set for it."
+  + " larger number means better layout result, but more memory 
consumer");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SORT_COLUMNS = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sort.columns")
+  .defaultValue("")
+  .sinceVersion("0.10.0")
+  .withDocumentation("sort columns for build curve optimize. default value 
is empty string which means no sort."
+  + " more sort columns you specify, the worse data layout result. No 
more than 4 are recommended");
+
+  public static final ConfigProperty DATA_LAYOUT_DATA_SKIPPING_ENABLE = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "data.skipping.enable")
+  .defaultValue("true")
+  .sinceVersion("0.10.0")
+  .withDocumentation("enable dataSkipping for hudi, when optimize 
finished, statistics will be collected which used for dataSkipping");
+
+  public static final ConfigProperty DATA_LAYOUT_DATA_STATISTICS_SAVE_MODE = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "statistics.save.mode")
+  .defaultValue("append")
+  .sinceVersion("0.10.0")
+  .withDocumentation("how to save statistics info every time, when do 
optimize");

Review comment:
   also what's the possible value?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-30 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieOptimizeConfig.java
##
@@ -0,0 +1,133 @@
+/*
+ * 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.config;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Hoodie Configs for Data layout optimize.
+ */
+public class HoodieOptimizeConfig extends HoodieConfig {
+  // Any Data layout optimize params can be saved with this prefix
+  public static final String DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX = 
"hoodie.data.layout.optimize.";
+  public static final ConfigProperty DATA_LAYOUT_STRATEGY = ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "strategy")
+  .defaultValue("z-order")
+  .sinceVersion("0.10.0")
+  .withDocumentation("config to provide a way to optimize data layout for 
table, current only support z-order and hilbert");
+
+  public static final ConfigProperty DATA_LAYOUT_BUILD_CURVE_METHOD = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "build.curve.optimize.method")
+  .defaultValue("directly")
+  .sinceVersion("0.10.0")
+  .withDocumentation("Config to provide whether use directly/sample method 
to build curve optimize for data layout,"
+  + " build curve_optimize by directly method is faster than by sample 
method, however sample method produce a better data layout");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SAMPLE_NUMBER 
= ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sample.number")
+  .defaultValue("20")
+  .sinceVersion("0.10.0")
+  .withDocumentation("when set" + DATA_LAYOUT_BUILD_CURVE_METHOD.key() + " 
to sample method, sample number need to be set for it."
+  + " larger number means better layout result, but more memory 
consumer");
+
+  public static final ConfigProperty DATA_LAYOUT_CURVE_OPTIMIZE_SORT_COLUMNS = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "curve.optimize.sort.columns")
+  .defaultValue("")
+  .sinceVersion("0.10.0")
+  .withDocumentation("sort columns for build curve optimize. default value 
is empty string which means no sort."
+  + " more sort columns you specify, the worse data layout result. No 
more than 4 are recommended");
+
+  public static final ConfigProperty DATA_LAYOUT_DATA_SKIPPING_ENABLE = 
ConfigProperty

Review comment:
   can be ConfigProperty




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-30 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieOptimizeConfig.java
##
@@ -0,0 +1,133 @@
+/*
+ * 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.config;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Hoodie Configs for Data layout optimize.
+ */
+public class HoodieOptimizeConfig extends HoodieConfig {
+  // Any Data layout optimize params can be saved with this prefix
+  public static final String DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX = 
"hoodie.data.layout.optimize.";
+  public static final ConfigProperty DATA_LAYOUT_STRATEGY = ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "strategy")
+  .defaultValue("z-order")
+  .sinceVersion("0.10.0")
+  .withDocumentation("config to provide a way to optimize data layout for 
table, current only support z-order and hilbert");
+
+  public static final ConfigProperty DATA_LAYOUT_BUILD_CURVE_METHOD = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "build.curve.optimize.method")

Review comment:
   build.curve.optimize.strategy would be better?

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieOptimizeConfig.java
##
@@ -0,0 +1,133 @@
+/*
+ * 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.config;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+
+import java.io.File;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Hoodie Configs for Data layout optimize.
+ */
+public class HoodieOptimizeConfig extends HoodieConfig {
+  // Any Data layout optimize params can be saved with this prefix
+  public static final String DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX = 
"hoodie.data.layout.optimize.";
+  public static final ConfigProperty DATA_LAYOUT_STRATEGY = ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "strategy")
+  .defaultValue("z-order")
+  .sinceVersion("0.10.0")
+  .withDocumentation("config to provide a way to optimize data layout for 
table, current only support z-order and hilbert");
+
+  public static final ConfigProperty DATA_LAYOUT_BUILD_CURVE_METHOD = 
ConfigProperty
+  .key(DATA_LAYOUT_OPTIMIZE_PARAM_PREFIX + "build.curve.optimize.method")
+  .defaultValue("directly")

Review comment:
   what's the possible value?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-28 Thread GitBox


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



##
File path: 
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java
##
@@ -229,6 +229,11 @@ public HoodieWriteMetadata 
deletePartitions(HoodieEngineContext context, String
 throw new HoodieNotSupportedException("DeletePartitions is not supported 
yet");
   }
 
+  @Override
+  public HoodieWriteMetadata> optimize(HoodieEngineContext 
context, String instantTime, List> records) {
+throw new HoodieNotSupportedException("optimize data layouy is not 
supported yet");

Review comment:
   nit:layout




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/Zoptimize.scala
##
@@ -0,0 +1,750 @@
+/*
+ * 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.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.spark.sql.hudi.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 {
+
+  case class FileStats(val minVal: String, val maxVal: String, val num_nulls: 
Int = 0)
+  case class ColumnFileStats(val fileName: String, val colName: String, val 
minVal: String, val maxVal: String, val num_nulls: Int = 0)
+
+  def createZIndexedDataFrameByRange(df: DataFrame, zCols: String, fileNum: 
Int): DataFrame = {
+createZIndexedDataFrameByRange(df, zCols.split(",").map(_.trim), fileNum)
+  }
+
+  def createZIndexDataFrameBySample(df: DataFrame, zCols: String, fileNum: 
Int): DataFrame = {
+createZIndexDataFrameBySample(df, zCols.split(",").map(_.trim), fileNum)
+  }
+
+  /**
+* create z-order DataFrame by sample
+* first, sample origin data to get z-cols bounds, then create z-order 
DataFrame
+* support all type data.
+* this method need more resource and cost more time than 
createZIndexedDataFrameByMapValue
+*/
+  def createZIndexDataFrameBySample(df: DataFrame, zCols: Seq[String], 
fileNum: Int): DataFrame = {
+val spark = df.sparkSession
+val columnsMap = df.schema.fields.map(item => (item.name, item)).toMap
+val fieldNum = df.schema.fields.length
+val checkCols = zCols.filter(col => columnsMap(col) != null)
+
+if (zCols.isEmpty || checkCols.isEmpty) {
+  df
+} else {
+  val zFields = zCols.map { col =>
+val newCol = columnsMap(col)
+if (newCol == null) {
+  (-1, null)
+} else {
+  newCol.dataType match {
+case LongType | DoubleType | FloatType | StringType | IntegerType 
| DateType | TimestampType | ShortType | ByteType =>
+  (df.schema.fields.indexOf(newCol), newCol)
+case d: DecimalType =>
+  (df.schema.fields.indexOf(newCol), newCol)
+case _ =>
+  (-1, null)
+  }
+}
+  }.filter(_._1 != -1)
+  // Complex type found, use createZIndexedDataFrameByRange
+  if (zFields.length != zCols.length) {
+return createZIndexedDataFrameByRange(df, zCols, fieldNum)
+  }
+
+  val rawRdd = df.rdd
+  val sampleRdd = rawRdd.map { row =>
+val values = zFields.map { case (index, field) =>
+  field.dataType match {
+case LongType =>
+   

[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/Zoptimize.scala
##
@@ -0,0 +1,750 @@
+/*
+ * 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.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.spark.sql.hudi.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 {
+
+  case class FileStats(val minVal: String, val maxVal: String, val num_nulls: 
Int = 0)
+  case class ColumnFileStats(val fileName: String, val colName: String, val 
minVal: String, val maxVal: String, val num_nulls: Int = 0)
+
+  def createZIndexedDataFrameByRange(df: DataFrame, zCols: String, fileNum: 
Int): DataFrame = {
+createZIndexedDataFrameByRange(df, zCols.split(",").map(_.trim), fileNum)
+  }
+
+  def createZIndexDataFrameBySample(df: DataFrame, zCols: String, fileNum: 
Int): DataFrame = {
+createZIndexDataFrameBySample(df, zCols.split(",").map(_.trim), fileNum)
+  }
+
+  /**
+* create z-order DataFrame by sample
+* first, sample origin data to get z-cols bounds, then create z-order 
DataFrame
+* support all type data.
+* this method need more resource and cost more time than 
createZIndexedDataFrameByMapValue
+*/
+  def createZIndexDataFrameBySample(df: DataFrame, zCols: Seq[String], 
fileNum: Int): DataFrame = {
+val spark = df.sparkSession
+val columnsMap = df.schema.fields.map(item => (item.name, item)).toMap
+val fieldNum = df.schema.fields.length
+val checkCols = zCols.filter(col => columnsMap(col) != null)
+
+if (zCols.isEmpty || checkCols.isEmpty) {
+  df
+} else {
+  val zFields = zCols.map { col =>
+val newCol = columnsMap(col)
+if (newCol == null) {
+  (-1, null)
+} else {
+  newCol.dataType match {
+case LongType | DoubleType | FloatType | StringType | IntegerType 
| DateType | TimestampType | ShortType | ByteType =>
+  (df.schema.fields.indexOf(newCol), newCol)
+case d: DecimalType =>
+  (df.schema.fields.indexOf(newCol), newCol)
+case _ =>
+  (-1, null)
+  }
+}
+  }.filter(_._1 != -1)
+  // Complex type found, use createZIndexedDataFrameByRange
+  if (zFields.length != zCols.length) {
+return createZIndexedDataFrameByRange(df, zCols, fieldNum)
+  }
+
+  val rawRdd = df.rdd
+  val sampleRdd = rawRdd.map { row =>
+val values = zFields.map { case (index, field) =>
+  field.dataType match {
+case LongType =>
+   

[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/OptimizeTableByCurve.scala
##
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.catalyst.expressions.{And, Expression, Literal, 
PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, 
Project}
+import org.apache.spark.sql.execution.command.RunnableCommand
+
+abstract class OptimizeTableByCurve(
+target: LogicalPlan,
+condition: Option[Expression],
+orderFields: Seq[Expression],
+outputFileNum: Option[Int],
+options: Option[Map[String, String]],
+curveName: String = "z-order") extends RunnableCommand with Logging with 
PredicateHelper {
+
+  val z_orderFields = 
orderFields.map(Zoptimize.getTargetColNameParts(_).mkString("."))
+
+  val optimizeCondition = condition.getOrElse(Literal.TrueLiteral)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val optimizeDF = curveName match {
+  case "z-order" =>
+buildZData(sparkSession)
+  // to do support hibert curve
+  case "hilbert" =>
+throw new SparkException(s"now not support hilbert optimize")
+  case other =>
+throw new SparkException(s"only support z-order curve but find: 
${other}")
+}
+doOptimize(optimizeDF, options, z_orderFields)
+Seq.empty[Row]
+  }
+
+  def buildZData(sparkSession: SparkSession): DataFrame = {
+
+val (partitionPredicates, dataPredicates) = 
splitPartitionAndDataPredicates(target, optimizeCondition, sparkSession)
+if (dataPredicates.nonEmpty) {
+  throw new SparkException(s"only support partitionPredicates for 
optimizer Curve," +
+s" but find other Predicates: 
${dataPredicates.map(_.toString()).mkString("::")} ")
+}
+val df = Dataset.ofRows(sparkSession, Project(target.output, 
Filter(partitionPredicates.reduce(And), target)))
+if (options.isDefined && 
options.get.getOrElse("hoodie.zorder.method.bySample", "false").toBoolean) {

Review comment:
   define constant




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/spark/sql/hudi/ZOrderingUtil.java
##
@@ -0,0 +1,228 @@
+/*
+ * 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;
+
+import sun.misc.Unsafe;
+
+import java.nio.charset.Charset;
+
+public class ZOrderingUtil {

Review comment:
   move to hudi-client-common module?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java
##
@@ -105,6 +106,19 @@ public 
SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context,
 validateWriteResult(writeMetadata);
 commitOnAutoCommit(writeMetadata);
 if (!writeMetadata.getCommitMetadata().isPresent()) {
+  WriteOperationType realOperationType = operationType;
+  Option> realExtraMetadata = extraMetadata;
+  if 
(config.getClusteringExecutionStrategyClass().equals("org.apache.hudi.clustering.SparkZSortAndSizeExecutionStrategy"))
 {
+realOperationType = WriteOperationType.ZORDER;
+Map metaDataMap = new HashMap<>();
+if (extraMetadata.isPresent()) {
+  metaDataMap = extraMetadata.get();
+}
+metaDataMap.put("_hoodie.zorder.cols", clusteringPlan.getStrategy()
+
.getStrategyParams().getOrDefault("hoodie.clustering.plan.strategy.sort.columns",
 ""));

Review comment:
   here why use clustering sort columns instead of defining 
hoodie.zorder.columns? is hoodie.zorder.cols an internal concept?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -182,6 +183,13 @@ public boolean commitStats(String instantTime, 
List stats, Opti
 try {
   preCommit(instantTime, metadata);
   commit(table, commitActionType, instantTime, metadata, stats);
+  if (extraMetadata.isPresent() && 
extraMetadata.get().containsKey("_hoodie.zorder.enable")) {
+String basePath = table.getMetaClient().getBasePath();
+String indexPath = table.getMetaClient().getZindexPath();
+List zFiles = stats.stream().map(s -> new Path(basePath, 
s.getPartitionPath()).toString()).collect(Collectors.toList());
+saveStatisticsInfo(zFiles,
+extraMetadata.get().getOrDefault("_hoodie.zorder.cols", ""), 
indexPath, "append");

Review comment:
   can define `hoodie.zorder.columns` in HoodieClusteringConfig?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -182,6 +183,13 @@ public boolean commitStats(String instantTime, 
List stats, Opti
 try {
   preCommit(instantTime, metadata);
   commit(table, commitActionType, instantTime, metadata, stats);
+  if (extraMetadata.isPresent() && 
extraMetadata.get().containsKey("_hoodie.zorder.enable")) {

Review comment:
   I am a little confused about the flag here, if we really need this flag, 
if user specify zorder columns and strategy, this should be true, and else 
false.




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
##
@@ -238,6 +238,17 @@ private synchronized FileSystemViewManager 
getViewManager() {
*/
   public abstract HoodieWriteMetadata 
insertOverwriteTable(HoodieEngineContext context, String instantTime, I 
records);
 
+  /**
+   * Replaces all the existing records of the Hoodie table and inserts z-sort 
records into Hoodie table at the supplied instantTime,
+   * 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 zorderWrite(HoodieEngineContext 
context, String instantTime, I records);

Review comment:
   please unify `zorder` and `zOrder`




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -182,6 +183,13 @@ public boolean commitStats(String instantTime, 
List stats, Opti
 try {
   preCommit(instantTime, metadata);
   commit(table, commitActionType, instantTime, metadata, stats);
+  if (extraMetadata.isPresent() && 
extraMetadata.get().containsKey("_hoodie.zorder.enable")) {

Review comment:
   can define constant in `HoodieClusteringConfig`?

##
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
##
@@ -182,6 +183,13 @@ public boolean commitStats(String instantTime, 
List stats, Opti
 try {
   preCommit(instantTime, metadata);
   commit(table, commitActionType, instantTime, metadata, stats);
+  if (extraMetadata.isPresent() && 
extraMetadata.get().containsKey("_hoodie.zorder.enable")) {
+String basePath = table.getMetaClient().getBasePath();
+String indexPath = table.getMetaClient().getZindexPath();
+List zFiles = stats.stream().map(s -> new Path(basePath, 
s.getPartitionPath()).toString()).collect(Collectors.toList());
+saveStatisticsInfo(zFiles,
+extraMetadata.get().getOrDefault("_hoodie.zorder.cols", ""), 
indexPath, "append");

Review comment:
   ditto




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



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

Review comment:
   can we move it to hudi-client-common module?




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/clustering/SparkZSortAndSizeExecutionStrategy.java
##
@@ -0,0 +1,142 @@
+/*
+ * 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.clustering;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import 
org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitioner;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
+import org.apache.hudi.table.HoodieSparkMergeOnReadTable;
+import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.Zoptimize$;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class SparkZSortAndSizeExecutionStrategy>
+extends SparkSortAndSizeExecutionStrategy {
+
+  public SparkZSortAndSizeExecutionStrategy(

Review comment:
   Keep the construction method is enough and remove the another one.




-- 
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-unsubscr...@hudi.apache.org

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




[GitHub] [hudi] leesf commented on a change in pull request #3330: [HUDI-2101][RFC-28]support z-order for hudi

2021-07-23 Thread GitBox


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



##
File path: 
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/clustering/SparkZSortAndSizeExecutionStrategy.java
##
@@ -0,0 +1,142 @@
+/*
+ * 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.clustering;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import 
org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.execution.bulkinsert.NonSortPartitioner;
+import org.apache.hudi.table.BulkInsertPartitioner;
+import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
+import org.apache.hudi.table.HoodieSparkMergeOnReadTable;
+import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.Zoptimize$;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+public class SparkZSortAndSizeExecutionStrategy>
+extends SparkSortAndSizeExecutionStrategy {
+
+  public SparkZSortAndSizeExecutionStrategy(

Review comment:
   Keep the construction method is enough?




-- 
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-unsubscr...@hudi.apache.org

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