yihua commented on a change in pull request #4106: URL: https://github.com/apache/hudi/pull/4106#discussion_r765187640
########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java ########## @@ -544,30 +548,27 @@ public static BuildCurveStrategyType fromValue(String value) { } /** - * strategy types for optimize layout for hudi data. + * Layout optimization strategies such us Z-order, Hilbert space curves, etc Review comment: nit: typo `us` ########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java ########## @@ -544,30 +548,27 @@ public static BuildCurveStrategyType fromValue(String value) { } /** - * strategy types for optimize layout for hudi data. + * Layout optimization strategies such us Z-order, Hilbert space curves, etc */ - public enum BuildLayoutOptimizationStrategy { + public enum LayoutOptimizationStrategy { ZORDER("z-order"), HILBERT("hilbert"); + + private static final Map<String, LayoutOptimizationStrategy> VALUE_ENUM_MAP = createValueToEnumMap(); + private final String value; - BuildLayoutOptimizationStrategy(String value) { + LayoutOptimizationStrategy(String value) { this.value = value; } - public String toCustomString() { - return value; + public static LayoutOptimizationStrategy fromValue(String value) { + return VALUE_ENUM_MAP.get(value); Review comment: Do we want to throw a Hoodie exception here if the enum cannot be found? ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java ########## @@ -621,4 +495,5 @@ public static String createIndexMergeSql( String.format("%s.%s", newIndexTable, columns.get(0)) ); } + Review comment: nit: remote empty line? ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java ########## @@ -402,52 +297,52 @@ public static void updateZIndexFor( .filter(f -> !completedCommits.contains(f)) .collect(Collectors.toList()); - Dataset<Row> finalZIndexDf; - - // Before writing out new version of the Z-index table we need to merge it + Dataset<Row> finalColStatsIndexDf; + + // Before writing out new version of the col-stats-index table we need to merge it // with the most recent one that were successfully persisted previously if (validIndexTables.isEmpty()) { - finalZIndexDf = newZIndexDf; + finalColStatsIndexDf = newColStatsIndexDf; } else { // NOTE: That Parquet schema might deviate from the original table schema (for ex, // by upcasting "short" to "integer" types, etc), and hence we need to re-adjust it // prior to merging, since merging might fail otherwise due to schemas incompatibility - finalZIndexDf = + finalColStatsIndexDf = tryMergeMostRecentIndexTableInto( sparkSession, - newZIndexDf, - // Load current most recent Z-index table + newColStatsIndexDf, + // Load current most recent col-stats-index table sparkSession.read().load( - new Path(zindexFolderPath, validIndexTables.get(validIndexTables.size() - 1)).toString() + new Path(indexFolderPath, validIndexTables.get(validIndexTables.size() - 1)).toString() ) ); // Clean up all index tables (after creation of the new index) tablesToCleanup.addAll(validIndexTables); } - // Persist new Z-index table - finalZIndexDf - .repartition(1) - .write() - .format("parquet") - .save(newIndexTablePath.toString()); + // Persist new col-stats-index table + finalColStatsIndexDf + .repartition(1) + .write() + .format("parquet") + .save(newIndexTablePath.toString()); Review comment: I see some changes are formatting only. Are they from IDE auto reformatting? ########## File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSpaceCurveLayoutOptimization.scala ########## @@ -119,106 +140,12 @@ class TestTableLayoutOptimization extends HoodieClientTestBase { ) } - def assertRowsMatch(one: DataFrame, other: DataFrame) = { + private def assertRowsMatch(one: DataFrame, other: DataFrame) = { val rows = one.count() assert(rows == other.count() && one.intersect(other).count() == rows) } - @Test Review comment: Are these tests obsolete or they should still be kept? ########## File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala ########## @@ -53,100 +43,25 @@ class TestZOrderLayoutOptimization extends HoodieClientTestBase { .add("c7", BinaryType) .add("c8", ByteType) - val commonOpts = Map( - "hoodie.insert.shuffle.parallelism" -> "4", - "hoodie.upsert.shuffle.parallelism" -> "4", - "hoodie.bulkinsert.shuffle.parallelism" -> "4", - DataSourceWriteOptions.RECORDKEY_FIELD.key() -> "_row_key", - DataSourceWriteOptions.PARTITIONPATH_FIELD.key() -> "partition", - DataSourceWriteOptions.PRECOMBINE_FIELD.key() -> "timestamp", - HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" - ) - @BeforeEach override def setUp() { initPath() initSparkContexts() - spark = sqlContext.sparkSession - initTestDataGenerator() initFileSystem() + spark = sqlContext.sparkSession } @AfterEach override def tearDown() = { - cleanupSparkContexts() - cleanupTestDataGenerator() cleanupFileSystem() - } - - @ParameterizedTest - @ValueSource(strings = Array("COPY_ON_WRITE", "MERGE_ON_READ")) - def testZOrderingLayoutClustering(tableType: String): Unit = { - val targetRecordsCount = 10000 - // Bulk Insert Operation - val records = recordsToStrings(dataGen.generateInserts("001", targetRecordsCount)).toList - val writeDf: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records, 2)) - - writeDf.write.format("org.apache.hudi") - .options(commonOpts) - .option("hoodie.compact.inline", "false") - .option(DataSourceWriteOptions.OPERATION.key(), DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) - .option(DataSourceWriteOptions.TABLE_TYPE.key(), tableType) - // option for clustering - .option("hoodie.parquet.small.file.limit", "0") - .option("hoodie.clustering.inline", "true") - .option("hoodie.clustering.inline.max.commits", "1") - .option("hoodie.clustering.plan.strategy.target.file.max.bytes", "1073741824") - .option("hoodie.clustering.plan.strategy.small.file.limit", "629145600") - .option("hoodie.clustering.plan.strategy.max.bytes.per.group", Long.MaxValue.toString) - .option("hoodie.clustering.plan.strategy.target.file.max.bytes", String.valueOf(64 * 1024 * 1024L)) - .option(HoodieClusteringConfig.LAYOUT_OPTIMIZE_ENABLE.key, "true") - .option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "begin_lat, begin_lon") - .mode(SaveMode.Overwrite) - .save(basePath) - - val hudiMetaClient = HoodieTableMetaClient.builder - .setConf(hadoopConf) - .setBasePath(basePath) - .setLoadActiveTimelineOnLoad(true) - .build - - val lastCommit = hudiMetaClient.getActiveTimeline.getAllCommitsTimeline.lastInstant().get() - - assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, lastCommit.getAction) - assertEquals(HoodieInstant.State.COMPLETED, lastCommit.getState) - - val readDf = - spark.read - .format("hudi") - .load(basePath) - - val readDfSkip = - spark.read - .option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "true") - .format("hudi") - .load(basePath) - - assertEquals(targetRecordsCount, readDf.count()) - assertEquals(targetRecordsCount, readDfSkip.count()) - - readDf.createOrReplaceTempView("hudi_snapshot_raw") - readDfSkip.createOrReplaceTempView("hudi_snapshot_skipping") - - def select(tableName: String) = - spark.sql(s"SELECT * FROM $tableName WHERE begin_lat >= 0.49 AND begin_lat < 0.51 AND begin_lon >= 0.49 AND begin_lon < 0.51") - - assertRowsMatch( - select("hudi_snapshot_raw"), - select("hudi_snapshot_skipping") - ) + cleanupSparkContexts() } @Test @Disabled Review comment: Should this be reenabled? ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveOptimizationSortPartitioner.java ########## @@ -74,23 +81,43 @@ public RDDSpatialCurveOptimizationSortPartitioner(HoodieSparkEngineContext spark private JavaRDD<GenericRecord> prepareGenericRecord(JavaRDD<HoodieRecord<T>> inputRecords, final int numOutputGroups, final Schema schema) { SerializableSchema serializableSchema = new SerializableSchema(schema); JavaRDD<GenericRecord> genericRecordJavaRDD = inputRecords.map(f -> (GenericRecord) f.getData().getInsertValue(serializableSchema.get()).get()); - Dataset<Row> originDF = AvroConversionUtils.createDataFrame(genericRecordJavaRDD.rdd(), schema.toString(), sparkEngineContext.getSqlContext().sparkSession()); - Dataset<Row> zDataFrame; + Dataset<Row> originDF = + AvroConversionUtils.createDataFrame( + genericRecordJavaRDD.rdd(), + schema.toString(), + sparkEngineContext.getSqlContext().sparkSession() + ); + + Dataset<Row> sortedDF = reorder(originDF, numOutputGroups); + + return HoodieSparkUtils.createRdd(sortedDF, schema.getName(), + schema.getNamespace(), false, org.apache.hudi.common.util.Option.empty()).toJavaRDD(); + } + + private Dataset<Row> reorder(Dataset<Row> sourceDF, int numOutputGroups) { + String orderedColumnsListConfig = config.getClusteringSortColumns(); + + if (isNullOrEmpty(orderedColumnsListConfig) || numOutputGroups <= 0) { + // No-op + return sourceDF; + } + + List<String> orderedCols = + Arrays.stream(orderedColumnsListConfig.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy = + HoodieClusteringConfig.LayoutOptimizationStrategy.fromValue(config.getLayoutOptimizationStrategy()); switch (config.getLayoutOptimizationCurveBuildMethod()) { case DIRECT: - zDataFrame = OrderingIndexHelper - .createOptimizedDataFrameByMapValue(originDF, config.getClusteringSortColumns(), numOutputGroups, config.getLayoutOptimizationStrategy()); - break; + return SpaceCurveSortingHelper.orderDataFrameByMappingValues(sourceDF, layoutOptStrategy, orderedCols, numOutputGroups); case SAMPLE: - zDataFrame = OrderingIndexHelper - .createOptimizeDataFrameBySample(originDF, config.getClusteringSortColumns(), numOutputGroups, config.getLayoutOptimizationStrategy()); - break; + return SpaceCurveSortingHelper.orderDataFrameBySamplingValues(sourceDF, layoutOptStrategy, orderedCols, numOutputGroups); default: throw new HoodieException("Not a valid build curve method for doWriteOperation: "); Review comment: nit: this can print the `config.getLayoutOptimizationCurveBuildMethod()` in the log message. ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java ########## @@ -79,13 +71,11 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import scala.collection.JavaConversions; - import static org.apache.hudi.util.DataTypeUtils.areCompatible; -public class ZOrderingIndexHelper { +public class ColumnStatsIndexHelper { Review comment: nit: (not a strong opinion) shall this be simply names as `ColumnStatsHelper`? ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/sort/SpaceCurveSortingHelper.java ########## @@ -0,0 +1,260 @@ +/* + * 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.sort; + +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.optimize.HilbertCurveUtils; +import org.apache.hudi.optimize.ZOrderingUtil; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.Row$; +import org.apache.spark.sql.hudi.execution.RangeSampleSort$; +import org.apache.spark.sql.hudi.execution.ZorderingBinarySort; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructType$; +import org.apache.spark.sql.types.TimestampType; +import org.davidmoten.hilbert.HilbertCurve; +import scala.collection.JavaConversions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class SpaceCurveSortingHelper { + + private static final Logger LOG = LogManager.getLogger(SpaceCurveSortingHelper.class); + + /** + * Orders provided {@link Dataset} by mapping values of the provided list of columns + * {@code orderByCols} onto a specified space curve (Z-curve, Hilbert, etc) + * + * <p/> + * NOTE: Only support base data-types: long,int,short,double,float,string,timestamp,decimal,date,byte. + * This method is more effective than {@link #orderDataFrameBySamplingValues} leveraging + * data sampling instead of direct mapping + * + * @param df Spark {@link Dataset} holding data to be ordered + * @param orderByCols list of columns to be ordered by + * @param targetPartitionCount target number of output partitions + * @param layoutOptStrategy target layout optimization strategy + * @return a {@link Dataset} holding data ordered by mapping tuple of values from provided columns + * onto a specified space-curve + */ + public static Dataset<Row> orderDataFrameByMappingValues( + Dataset<Row> df, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + List<String> orderByCols, + int targetPartitionCount + ) { + Map<String, StructField> columnsMap = + Arrays.stream(df.schema().fields()) + .collect(Collectors.toMap(StructField::name, Function.identity())); + + List<String> checkCols = + orderByCols.stream() + .filter(columnsMap::containsKey) + .collect(Collectors.toList()); + + if (orderByCols.size() != checkCols.size()) { + LOG.error(String.format("Trying to ordering over a column(s) not present in the schema (%s); skipping", CollectionUtils.diff(orderByCols, checkCols))); + return df; + } Review comment: Does this check work with nested columns? ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/columnstats/ColumnStatsIndexHelper.java ########## @@ -313,66 +207,67 @@ public static String getNumNullsColumnNameFor(String colName) { }) .filter(Objects::nonNull); - StructType indexSchema = composeIndexSchema(zorderedColumnSchemas); + StructType indexSchema = composeIndexSchema(orderedColumnSchemas); return sparkSession.createDataFrame(allMetaDataRDD, indexSchema); } /** * <p/> - * Updates state of the Z-index by: + * Updates state of the column-stats index by: * <ol> - * <li>Updating Z-index with statistics for {@code sourceBaseFiles}, collecting corresponding - * column statistics from Parquet footers</li> - * <li>Merging newly built Z-index table with the most recent one (if present and not preempted)</li> + * <li>Updating column-stats index with statistics for {@code sourceBaseFiles}, + * collecting corresponding column statistics from Parquet footers</li> + * <li>Merging newly built column-stats index table with the most recent one (if present + * and not preempted)</li> * <li>Cleans up any residual index tables, that weren't cleaned up before</li> * </ol> * * @param sparkSession encompassing Spark session * @param sourceTableSchema instance of {@link StructType} bearing source table's writer's schema * @param sourceBaseFiles list of base-files to be indexed - * @param zorderedCols target Z-ordered columns - * @param zindexFolderPath Z-index folder path + * @param orderedCols target ordered columns + * @param indexFolderPath col-stats index folder path * @param commitTime current operation commit instant * @param completedCommits all previously completed commit instants */ - public static void updateZIndexFor( + public static void updateColumnStatsIndexFor( Review comment: nit: if you decide to go with `ColumnStatsHelper`, this can be renames as `updateColumnStatsFor()`. ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveOptimizationSortPartitioner.java ########## @@ -74,23 +81,43 @@ public RDDSpatialCurveOptimizationSortPartitioner(HoodieSparkEngineContext spark private JavaRDD<GenericRecord> prepareGenericRecord(JavaRDD<HoodieRecord<T>> inputRecords, final int numOutputGroups, final Schema schema) { SerializableSchema serializableSchema = new SerializableSchema(schema); JavaRDD<GenericRecord> genericRecordJavaRDD = inputRecords.map(f -> (GenericRecord) f.getData().getInsertValue(serializableSchema.get()).get()); - Dataset<Row> originDF = AvroConversionUtils.createDataFrame(genericRecordJavaRDD.rdd(), schema.toString(), sparkEngineContext.getSqlContext().sparkSession()); - Dataset<Row> zDataFrame; + Dataset<Row> originDF = + AvroConversionUtils.createDataFrame( + genericRecordJavaRDD.rdd(), + schema.toString(), + sparkEngineContext.getSqlContext().sparkSession() + ); + + Dataset<Row> sortedDF = reorder(originDF, numOutputGroups); + + return HoodieSparkUtils.createRdd(sortedDF, schema.getName(), + schema.getNamespace(), false, org.apache.hudi.common.util.Option.empty()).toJavaRDD(); + } + + private Dataset<Row> reorder(Dataset<Row> sourceDF, int numOutputGroups) { Review comment: nit: we can pick one naming here, `originDF` vs `sourceDF` ########## File path: hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSpaceCurveLayoutOptimization.scala ########## @@ -52,23 +61,25 @@ class TestTableLayoutOptimization extends HoodieClientTestBase { HoodieWriteConfig.TBL_NAME.key -> "hoodie_test" ) - @BeforeEach override def setUp() { + @BeforeEach + override def setUp() { initPath() initSparkContexts() spark = sqlContext.sparkSession initTestDataGenerator() initFileSystem() } - @AfterEach override def tearDown() = { + @AfterEach + override def tearDown() = { cleanupSparkContexts() cleanupTestDataGenerator() cleanupFileSystem() } @ParameterizedTest - @MethodSource(Array("testLayOutParameter")) - def testOptimizewithClustering(tableType: String, optimizeMode: String): Unit = { + @MethodSource(Array("testLayoutOptimizationParameters")) + def testZOrderingLayoutClustering(tableType: String): Unit = { Review comment: nit: rename to `testClusteringWithLayoutOptimization`? ########## File path: hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java ########## @@ -79,7 +79,7 @@ public static final String AUXILIARYFOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".aux"; public static final String BOOTSTRAP_INDEX_ROOT_FOLDER_PATH = AUXILIARYFOLDER_NAME + Path.SEPARATOR + ".bootstrap"; public static final String HEARTBEAT_FOLDER_NAME = METAFOLDER_NAME + Path.SEPARATOR + ".heartbeat"; - public static final String ZINDEX_NAME = ".zindex"; + public static final String COLUMN_STATISTICS_INDEX_NAME = ".colstatsindex"; Review comment: nit: similar here. Your call on `colstatsindex` vs `colstats`. ########## File path: hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/sort/SpaceCurveSortingHelper.java ########## @@ -0,0 +1,260 @@ +/* + * 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.sort; + +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.config.HoodieClusteringConfig; +import org.apache.hudi.optimize.HilbertCurveUtils; +import org.apache.hudi.optimize.ZOrderingUtil; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.Row$; +import org.apache.spark.sql.hudi.execution.RangeSampleSort$; +import org.apache.spark.sql.hudi.execution.ZorderingBinarySort; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.StructType$; +import org.apache.spark.sql.types.TimestampType; +import org.davidmoten.hilbert.HilbertCurve; +import scala.collection.JavaConversions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class SpaceCurveSortingHelper { + + private static final Logger LOG = LogManager.getLogger(SpaceCurveSortingHelper.class); + + /** + * Orders provided {@link Dataset} by mapping values of the provided list of columns + * {@code orderByCols} onto a specified space curve (Z-curve, Hilbert, etc) + * + * <p/> + * NOTE: Only support base data-types: long,int,short,double,float,string,timestamp,decimal,date,byte. + * This method is more effective than {@link #orderDataFrameBySamplingValues} leveraging + * data sampling instead of direct mapping + * + * @param df Spark {@link Dataset} holding data to be ordered + * @param orderByCols list of columns to be ordered by + * @param targetPartitionCount target number of output partitions + * @param layoutOptStrategy target layout optimization strategy + * @return a {@link Dataset} holding data ordered by mapping tuple of values from provided columns + * onto a specified space-curve + */ + public static Dataset<Row> orderDataFrameByMappingValues( + Dataset<Row> df, + HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy, + List<String> orderByCols, + int targetPartitionCount + ) { + Map<String, StructField> columnsMap = + Arrays.stream(df.schema().fields()) + .collect(Collectors.toMap(StructField::name, Function.identity())); + + List<String> checkCols = + orderByCols.stream() + .filter(columnsMap::containsKey) + .collect(Collectors.toList()); + + if (orderByCols.size() != checkCols.size()) { + LOG.error(String.format("Trying to ordering over a column(s) not present in the schema (%s); skipping", CollectionUtils.diff(orderByCols, checkCols))); + return df; + } + + // In case when there's just one column to be ordered by, we can skip space-curve + // ordering altogether (since it will match linear ordering anyway) + if (orderByCols.size() == 1) { + String orderByColName = orderByCols.get(0); + LOG.debug(String.format("Single column to order by (%s), skipping space-curve ordering", orderByColName)); + + // TODO validate if we need Spark to re-partition + return df.repartitionByRange(targetPartitionCount, new Column(orderByColName)); + } + + int fieldNum = df.schema().fields().length; + + Map<Integer, StructField> fieldMap = + orderByCols.stream() + .collect( + Collectors.toMap(e -> Arrays.asList(df.schema().fields()).indexOf(columnsMap.get(e)), columnsMap::get)); + + JavaRDD<Row> sortedRDD; + switch (layoutOptStrategy) { + case ZORDER: + sortedRDD = createZCurveSortedRDD(df.toJavaRDD(), fieldMap, fieldNum, targetPartitionCount); + break; + case HILBERT: + sortedRDD = createHilbertSortedRDD(df.toJavaRDD(), fieldMap, fieldNum, targetPartitionCount); + break; + default: + throw new IllegalArgumentException(String.format("new only support z-order/hilbert optimize but find: %s", layoutOptStrategy)); + } + + // Compose new {@code StructType} for ordered RDDs + StructType newStructType = composeOrderedRDDStructType(df.schema()); + + return df.sparkSession() + .createDataFrame(sortedRDD, newStructType) + .drop("Index"); + } + + private static StructType composeOrderedRDDStructType(StructType schema) { + return StructType$.MODULE$.apply( + CollectionUtils.combine( + Arrays.asList(schema.fields()), + Arrays.asList(new StructField("Index", BinaryType$.MODULE$, true, Metadata.empty())) + ) + ); + } + + private static JavaRDD<Row> createZCurveSortedRDD(JavaRDD<Row> originRDD, Map<Integer, StructField> fieldMap, int fieldNum, int fileNum) { + return originRDD.map(row -> { + List<byte[]> zBytesList = fieldMap.entrySet().stream().map(entry -> { + int index = entry.getKey(); + StructField field = entry.getValue(); + DataType dataType = field.dataType(); + if (dataType instanceof LongType) { + return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getLong(index)); + } else if (dataType instanceof DoubleType) { + return ZOrderingUtil.doubleTo8Byte(row.isNullAt(index) ? Double.MAX_VALUE : row.getDouble(index)); + } else if (dataType instanceof IntegerType) { + return ZOrderingUtil.intTo8Byte(row.isNullAt(index) ? Integer.MAX_VALUE : row.getInt(index)); + } else if (dataType instanceof FloatType) { + return ZOrderingUtil.doubleTo8Byte(row.isNullAt(index) ? Float.MAX_VALUE : row.getFloat(index)); + } else if (dataType instanceof StringType) { + return ZOrderingUtil.utf8To8Byte(row.isNullAt(index) ? "" : row.getString(index)); + } else if (dataType instanceof DateType) { + return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getDate(index).getTime()); + } else if (dataType instanceof TimestampType) { + return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getTimestamp(index).getTime()); + } else if (dataType instanceof ByteType) { + return ZOrderingUtil.byteTo8Byte(row.isNullAt(index) ? Byte.MAX_VALUE : row.getByte(index)); + } else if (dataType instanceof ShortType) { + return ZOrderingUtil.intTo8Byte(row.isNullAt(index) ? Short.MAX_VALUE : row.getShort(index)); + } else if (dataType instanceof DecimalType) { + return ZOrderingUtil.longTo8Byte(row.isNullAt(index) ? Long.MAX_VALUE : row.getDecimal(index).longValue()); + } else if (dataType instanceof BooleanType) { + boolean value = row.isNullAt(index) ? false : row.getBoolean(index); + return ZOrderingUtil.intTo8Byte(value ? 1 : 0); + } else if (dataType instanceof BinaryType) { + return ZOrderingUtil.paddingTo8Byte(row.isNullAt(index) ? new byte[] {0} : (byte[]) row.get(index)); + } + return null; + }).filter(f -> f != null).collect(Collectors.toList()); + byte[][] zBytes = new byte[zBytesList.size()][]; + for (int i = 0; i < zBytesList.size(); i++) { + zBytes[i] = zBytesList.get(i); + } + List<Object> zVaules = new ArrayList<>(); + zVaules.addAll(scala.collection.JavaConverters.bufferAsJavaListConverter(row.toSeq().toBuffer()).asJava()); + zVaules.add(ZOrderingUtil.interleaving(zBytes, 8)); + return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(zVaules)); + }) + .sortBy(f -> new ZorderingBinarySort((byte[]) f.get(fieldNum)), true, fileNum); + } + + private static JavaRDD<Row> createHilbertSortedRDD(JavaRDD<Row> originRDD, Map<Integer, StructField> fieldMap, int fieldNum, int fileNum) { + return originRDD.mapPartitions(rows -> { + HilbertCurve hilbertCurve = HilbertCurve.bits(63).dimensions(fieldMap.size()); + return new Iterator<Row>() { + + @Override + public boolean hasNext() { + return rows.hasNext(); + } + + @Override + public Row next() { + Row row = rows.next(); + List<Long> longList = fieldMap.entrySet().stream().map(entry -> { + int index = entry.getKey(); + StructField field = entry.getValue(); + DataType dataType = field.dataType(); + if (dataType instanceof LongType) { + return row.isNullAt(index) ? Long.MAX_VALUE : row.getLong(index); + } else if (dataType instanceof DoubleType) { + return row.isNullAt(index) ? Long.MAX_VALUE : Double.doubleToLongBits(row.getDouble(index)); + } else if (dataType instanceof IntegerType) { + return row.isNullAt(index) ? Long.MAX_VALUE : (long)row.getInt(index); + } else if (dataType instanceof FloatType) { + return row.isNullAt(index) ? Long.MAX_VALUE : Double.doubleToLongBits((double) row.getFloat(index)); + } else if (dataType instanceof StringType) { + return row.isNullAt(index) ? Long.MAX_VALUE : ZOrderingUtil.convertStringToLong(row.getString(index)); Review comment: `createHilbertSortedRDD` also uses `ZOrderingUtil` APIs. So rename `ZOrderingUtil` to be general? During my testing of I'm confused by the Spark stages -- 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