1
0

[HUDI-2101][RFC-28] support z-order for hudi (#3330)

* [HUDI-2101]support z-order for hudi

* Renaming some configs for consistency/simplicity.

* Minor code cleanups

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
xiarixiaoyao
2021-11-03 00:31:57 +08:00
committed by GitHub
parent f9bc3e03e5
commit d194643b49
22 changed files with 2140 additions and 10 deletions

View File

@@ -380,6 +380,10 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime));
finalizeWrite(table, clusteringCommitTime, writeStats);
try {
// try to save statistics info to hudi
if (config.isDataSkippingEnabled() && config.isLayoutOptimizationEnabled() && !config.getClusteringSortColumns().isEmpty()) {
table.updateStatistics(context, writeStats, clusteringCommitTime, true);
}
LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata);
table.getActiveTimeline().transitionReplaceInflightToComplete(
HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime),

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.client.clustering.run.strategy;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieRecord;
@@ -28,6 +29,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
import org.apache.hudi.execution.bulkinsert.RDDSpatialCurveOptimizationSortPartitioner;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
@@ -77,7 +79,11 @@ public class SparkSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
* Create BulkInsertPartitioner based on strategy params.
*/
protected Option<BulkInsertPartitioner<T>> getPartitioner(Map<String, String> strategyParams, Schema schema) {
if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) {
if (getWriteConfig().isLayoutOptimizationEnabled()) {
// sort input records by z-order/hilbert
return Option.of(new RDDSpatialCurveOptimizationSortPartitioner((HoodieSparkEngineContext) getEngineContext(),
getWriteConfig(), HoodieAvroUtils.addMetadataFields(schema)));
} else if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) {
return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","),
HoodieAvroUtils.addMetadataFields(schema)));
} else {

View File

@@ -0,0 +1,98 @@
/*
* 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.execution.bulkinsert;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.HoodieSparkUtils;
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.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.ZCurveOptimizeHelper;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
/**
* A partitioner that does spartial curve optimization sorting based on specified column values for each RDD partition.
* support z-curve optimization, hilbert will come soon.
* @param <T> HoodieRecordPayload type
*/
public class RDDSpatialCurveOptimizationSortPartitioner<T extends HoodieRecordPayload>
implements BulkInsertPartitioner<JavaRDD<HoodieRecord<T>>> {
private final HoodieSparkEngineContext sparkEngineContext;
private final SerializableSchema serializableSchema;
private final HoodieWriteConfig config;
public RDDSpatialCurveOptimizationSortPartitioner(HoodieSparkEngineContext sparkEngineContext, HoodieWriteConfig config, Schema schema) {
this.sparkEngineContext = sparkEngineContext;
this.config = config;
this.serializableSchema = new SerializableSchema(schema);
}
@Override
public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
String payloadClass = config.getPayloadClass();
// do sort
JavaRDD<GenericRecord> preparedRecord = prepareGenericRecord(records, outputSparkPartitions, serializableSchema.get());
return preparedRecord.map(record -> {
String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieKey hoodieKey = new HoodieKey(key, partition);
HoodieRecordPayload avroPayload = ReflectionUtils.loadPayload(payloadClass,
new Object[] {Option.of(record)}, Option.class);
HoodieRecord hoodieRecord = new HoodieRecord(hoodieKey, avroPayload);
return hoodieRecord;
});
}
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;
switch (config.getLayoutOptimizationCurveBuildMethod()) {
case DIRECT:
zDataFrame = ZCurveOptimizeHelper.createZIndexedDataFrameByMapValue(originDF, config.getClusteringSortColumns(), numOutputGroups);
break;
case SAMPLE:
zDataFrame = ZCurveOptimizeHelper.createZIndexedDataFrameBySample(originDF, config.getClusteringSortColumns(), numOutputGroups);
break;
default:
throw new HoodieException("Not a valid build curve method for doWriteOperation: ");
}
return HoodieSparkUtils.createRdd(zDataFrame, schema.getName(),
schema.getNamespace(), false, org.apache.hudi.common.util.Option.empty()).toJavaRDD();
}
@Override
public boolean arePartitionRecordsSorted() {
return true;
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.table;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
@@ -34,6 +35,7 @@ import org.apache.hudi.common.model.HoodieBaseFile;
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.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
@@ -71,13 +73,16 @@ import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor;
import org.apache.hudi.table.action.savepoint.SavepointActionExecutor;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.ZCurveOptimizeHelper;
import org.apache.spark.api.java.JavaRDD;
import scala.collection.JavaConversions;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Implementation of a very heavily read-optimized Hoodie Table where, all data is stored in base files, with
@@ -152,6 +157,32 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload>
return new SparkInsertOverwriteTableCommitActionExecutor(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.getClusteringSortColumns();
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;
ZCurveOptimizeHelper.saveStatisticsInfo(sparkEngineContext
.getSqlContext().sparkSession().read().load(JavaConversions.asScalaBuffer(touchFiles)),
cols, indexPath, instantTime, validateCommits);
LOG.info(String.format("save statistic info sucessfully at commitTime: %s", instantTime));
}
@Override
public Option<HoodieCompactionPlan> scheduleCompaction(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");

View File

@@ -69,7 +69,7 @@ public class SparkBulkInsertHelper<T extends HoodieRecordPayload, R> extends Abs
//transition bulk_insert state to inflight
table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED,
table.getMetaClient().getCommitActionType(), instantTime), Option.empty(),
executor.getCommitActionType(), instantTime), Option.empty(),
config.shouldAllowMultiWriteOnSameInstant());
// write new files
JavaRDD<WriteStatus> writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism(), false);

View File

@@ -0,0 +1,355 @@
/*
* 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;
import scala.collection.JavaConversions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.HoodieSparkUtils$;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.optimize.ZOrderingUtil;
import org.apache.parquet.io.api.Binary;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.Row$;
import org.apache.spark.sql.SparkSession;
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.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.StringType$;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType$;
import org.apache.spark.sql.types.TimestampType;
import org.apache.spark.util.SerializableConfiguration;
import java.io.IOException;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class ZCurveOptimizeHelper {
private static final String SPARK_JOB_DESCRIPTION = "spark.job.description";
/**
* Create z-order DataFrame directly
* first, map all base type data to byte[8], then create z-order DataFrame
* only support base type data. long,int,short,double,float,string,timestamp,decimal,date,byte
* this method is more effective than createZIndexDataFrameBySample
*
* @param df a spark DataFrame holds parquet files to be read.
* @param zCols z-sort cols
* @param fileNum spark partition num
* @return a dataFrame sorted by z-order.
*/
public static Dataset<Row> createZIndexedDataFrameByMapValue(Dataset<Row> df, List<String> zCols, int fileNum) {
Map<String, StructField> columnsMap = Arrays.stream(df.schema().fields()).collect(Collectors.toMap(e -> e.name(), e -> e));
int fieldNum = df.schema().fields().length;
List<String> checkCols = zCols.stream().filter(f -> columnsMap.containsKey(f)).collect(Collectors.toList());
if (zCols.size() != checkCols.size()) {
return df;
}
// only one col to sort, no need to use z-order
if (zCols.size() == 1) {
return df.repartitionByRange(fieldNum, org.apache.spark.sql.functions.col(zCols.get(0)));
}
Map<Integer, StructField> fieldMap = zCols
.stream().collect(Collectors.toMap(e -> Arrays.asList(df.schema().fields()).indexOf(columnsMap.get(e)), e -> columnsMap.get(e)));
// z-sort
JavaRDD<Row> sortedRdd = df.toJavaRDD().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);
// create new StructType
List<StructField> newFields = new ArrayList<>();
newFields.addAll(Arrays.asList(df.schema().fields()));
newFields.add(new StructField("zIndex", BinaryType$.MODULE$, true, Metadata.empty()));
// create new DataFrame
return df.sparkSession().createDataFrame(sortedRdd, StructType$.MODULE$.apply(newFields)).drop("zIndex");
}
public static Dataset<Row> createZIndexedDataFrameByMapValue(Dataset<Row> df, String zCols, int fileNum) {
if (zCols == null || zCols.isEmpty() || fileNum <= 0) {
return df;
}
return createZIndexedDataFrameByMapValue(df,
Arrays.stream(zCols.split(",")).map(f -> f.trim()).collect(Collectors.toList()), fileNum);
}
public static Dataset<Row> createZIndexedDataFrameBySample(Dataset<Row> df, List<String> zCols, int fileNum) {
return RangeSampleSort$.MODULE$.sortDataFrameBySample(df, JavaConversions.asScalaBuffer(zCols), fileNum);
}
public static Dataset<Row> createZIndexedDataFrameBySample(Dataset<Row> df, String zCols, int fileNum) {
if (zCols == null || zCols.isEmpty() || fileNum <= 0) {
return df;
}
return createZIndexedDataFrameBySample(df, Arrays.stream(zCols.split(",")).map(f -> f.trim()).collect(Collectors.toList()), fileNum);
}
/**
* Parse min/max statistics stored in parquet footers for z-sort cols.
* no support collect statistics from timeStampType, since parquet file has not collect the statistics for timeStampType.
* to do adapt for rfc-27
*
* @param df a spark DataFrame holds parquet files to be read.
* @param cols z-sort cols
* @return a dataFrame holds all statistics info.
*/
public static Dataset<Row> getMinMaxValue(Dataset<Row> df, List<String> cols) {
Map<String, DataType> columnsMap = Arrays.stream(df.schema().fields()).collect(Collectors.toMap(e -> e.name(), e -> e.dataType()));
List<String> scanFiles = Arrays.asList(df.inputFiles());
SparkContext sc = df.sparkSession().sparkContext();
JavaSparkContext jsc = new JavaSparkContext(sc);
SerializableConfiguration serializableConfiguration = new SerializableConfiguration(sc.hadoopConfiguration());
int numParallelism = (scanFiles.size() / 3 + 1);
List<HoodieColumnRangeMetadata<Comparable>> colMinMaxInfos = new ArrayList<>();
String previousJobDescription = sc.getLocalProperty(SPARK_JOB_DESCRIPTION);
try {
String description = "Listing parquet column statistics";
jsc.setJobDescription(description);
colMinMaxInfos = jsc.parallelize(scanFiles, numParallelism).mapPartitions(paths -> {
Configuration conf = serializableConfiguration.value();
ParquetUtils parquetUtils = (ParquetUtils) BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
List<Collection<HoodieColumnRangeMetadata<Comparable>>> results = new ArrayList<>();
while (paths.hasNext()) {
String path = paths.next();
results.add(parquetUtils.readRangeFromParquetMetadata(conf, new Path(path), cols));
}
return results.stream().flatMap(f -> f.stream()).iterator();
}).collect();
} finally {
jsc.setJobDescription(previousJobDescription);
}
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> fileToStatsListMap = colMinMaxInfos.stream().collect(Collectors.groupingBy(e -> e.getFilePath()));
JavaRDD<Row> allMetaDataRDD = jsc.parallelize(fileToStatsListMap.values().stream().collect(Collectors.toList()), 1).map(f -> {
int colSize = f.size();
if (colSize == 0) {
return null;
} else {
List<Object> rows = new ArrayList<>();
rows.add(f.get(0).getFilePath());
cols.stream().forEach(col -> {
HoodieColumnRangeMetadata<Comparable> currentColRangeMetaData =
f.stream().filter(s -> s.getColumnName().trim().equalsIgnoreCase(col)).findFirst().orElse(null);
DataType colType = columnsMap.get(col);
if (currentColRangeMetaData == null || colType == null) {
throw new HoodieException(String.format("cannot collect min/max statistics for col: %s", col));
}
if (colType instanceof IntegerType) {
rows.add(currentColRangeMetaData.getMinValue());
rows.add(currentColRangeMetaData.getMaxValue());
} else if (colType instanceof DoubleType) {
rows.add(currentColRangeMetaData.getMinValue());
rows.add(currentColRangeMetaData.getMaxValue());
} else if (colType instanceof StringType) {
String minString = new String(((Binary)currentColRangeMetaData.getMinValue()).getBytes());
String maxString = new String(((Binary)currentColRangeMetaData.getMaxValue()).getBytes());
rows.add(minString);
rows.add(maxString);
} else if (colType instanceof DecimalType) {
Double minDecimal = Double.parseDouble(currentColRangeMetaData.getStringifier().stringify(Long.valueOf(currentColRangeMetaData.getMinValue().toString())));
Double maxDecimal = Double.parseDouble(currentColRangeMetaData.getStringifier().stringify(Long.valueOf(currentColRangeMetaData.getMaxValue().toString())));
rows.add(BigDecimal.valueOf(minDecimal));
rows.add(BigDecimal.valueOf(maxDecimal));
} else if (colType instanceof DateType) {
rows.add(java.sql.Date.valueOf(currentColRangeMetaData.getStringifier().stringify((int)currentColRangeMetaData.getMinValue())));
rows.add(java.sql.Date.valueOf(currentColRangeMetaData.getStringifier().stringify((int)currentColRangeMetaData.getMaxValue())));
} else if (colType instanceof LongType) {
rows.add(currentColRangeMetaData.getMinValue());
rows.add(currentColRangeMetaData.getMaxValue());
} else if (colType instanceof ShortType) {
rows.add(Short.parseShort(currentColRangeMetaData.getMinValue().toString()));
rows.add(Short.parseShort(currentColRangeMetaData.getMaxValue().toString()));
} else if (colType instanceof FloatType) {
rows.add(currentColRangeMetaData.getMinValue());
rows.add(currentColRangeMetaData.getMaxValue());
} else if (colType instanceof BinaryType) {
rows.add(((Binary)currentColRangeMetaData.getMinValue()).getBytes());
rows.add(((Binary)currentColRangeMetaData.getMaxValue()).getBytes());
} else if (colType instanceof BooleanType) {
rows.add(currentColRangeMetaData.getMinValue());
rows.add(currentColRangeMetaData.getMaxValue());
} else if (colType instanceof ByteType) {
rows.add(Byte.valueOf(currentColRangeMetaData.getMinValue().toString()));
rows.add(Byte.valueOf(currentColRangeMetaData.getMaxValue().toString()));
} else {
throw new HoodieException(String.format("Not support type: %s", colType));
}
rows.add(currentColRangeMetaData.getNumNulls());
});
return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(rows));
}
}).filter(f -> f != null);
List<StructField> allMetaDataSchema = new ArrayList<>();
allMetaDataSchema.add(new StructField("file", StringType$.MODULE$, true, Metadata.empty()));
cols.forEach(col -> {
allMetaDataSchema.add(new StructField(col + "_minValue", columnsMap.get(col), true, Metadata.empty()));
allMetaDataSchema.add(new StructField(col + "_maxValue", columnsMap.get(col), true, Metadata.empty()));
allMetaDataSchema.add(new StructField(col + "_num_nulls", LongType$.MODULE$, true, Metadata.empty()));
});
return df.sparkSession().createDataFrame(allMetaDataRDD, StructType$.MODULE$.apply(allMetaDataSchema));
}
public static Dataset<Row> getMinMaxValue(Dataset<Row> df, String cols) {
List<String> rawCols = Arrays.asList(cols.split(",")).stream().map(f -> f.trim()).collect(Collectors.toList());
return getMinMaxValue(df, rawCols);
}
/**
* Update statistics info.
* this method will update old index table by full out join,
* and save the updated table into a new index table based on commitTime.
* old index table will be cleaned also.
*
* @param df a spark DataFrame holds parquet files to be read.
* @param cols z-sort cols.
* @param indexPath index store path.
* @param commitTime current operation commitTime.
* @param validateCommits all validate commits for current table.
* @return
*/
public static void saveStatisticsInfo(Dataset<Row> df, String cols, String indexPath, String commitTime, List<String> validateCommits) {
Path savePath = new Path(indexPath, commitTime);
SparkSession spark = df.sparkSession();
FileSystem fs = FSUtils.getFs(indexPath, spark.sparkContext().hadoopConfiguration());
Dataset<Row> statisticsDF = ZCurveOptimizeHelper.getMinMaxValue(df, cols);
// try to find last validate index table from index path
try {
if (fs.exists(new Path(indexPath))) {
List<String> allIndexTables = Arrays
.stream(fs.listStatus(new Path(indexPath))).filter(f -> f.isDirectory()).map(f -> f.getPath().getName()).collect(Collectors.toList());
List<String> candidateIndexTables = allIndexTables.stream().filter(f -> validateCommits.contains(f)).sorted().collect(Collectors.toList());
List<String> residualTables = allIndexTables.stream().filter(f -> !validateCommits.contains(f)).collect(Collectors.toList());
Option<Dataset> latestIndexData = Option.empty();
if (!candidateIndexTables.isEmpty()) {
latestIndexData = Option.of(spark.read().load(new Path(indexPath, candidateIndexTables.get(candidateIndexTables.size() - 1)).toString()));
// clean old index table, keep at most 1 index table.
candidateIndexTables.remove(candidateIndexTables.size() - 1);
candidateIndexTables.forEach(f -> {
try {
fs.delete(new Path(indexPath, f));
} catch (IOException ie) {
throw new HoodieException(ie);
}
});
}
// clean residualTables
// retried cluster operations at the same instant time is also considered,
// the residual files produced by retried are cleaned up before save statistics
// save statistics info to index table which named commitTime
residualTables.forEach(f -> {
try {
fs.delete(new Path(indexPath, f));
} catch (IOException ie) {
throw new HoodieException(ie);
}
});
if (latestIndexData.isPresent() && latestIndexData.get().schema().equals(statisticsDF.schema())) {
// update the statistics info
String originalTable = "indexTable_" + java.util.UUID.randomUUID().toString().replace("-", "");
String updateTable = "updateTable_" + java.util.UUID.randomUUID().toString().replace("-", "");
latestIndexData.get().registerTempTable(originalTable);
statisticsDF.registerTempTable(updateTable);
// update table by full out join
List columns = Arrays.asList(statisticsDF.schema().fieldNames());
spark.sql(HoodieSparkUtils$
.MODULE$.createMergeSql(originalTable, updateTable, JavaConversions.asScalaBuffer(columns))).repartition(1).write().save(savePath.toString());
}
} else {
statisticsDF.repartition(1).write().mode("overwrite").save(savePath.toString());
}
} catch (IOException e) {
throw new HoodieException(e);
}
}
}

View File

@@ -19,6 +19,7 @@
package org.apache.hudi
import java.util.Properties
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.fs.{FileSystem, Path}
@@ -35,6 +36,7 @@ import org.apache.spark.sql.avro.SchemaConverters
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal}
import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.sql.{DataFrame, SparkSession}
@@ -283,4 +285,43 @@ object HoodieSparkUtils extends SparkAdapterSupport {
s"${tableSchema.fieldNames.mkString(",")}")
AttributeReference(columnName, field.get.dataType, field.get.nullable)()
}
/**
* Create merge sql to merge leftTable and right table.
*
* @param leftTable table name.
* @param rightTable table name.
* @param cols merged cols.
* @return merge sql.
*/
def createMergeSql(leftTable: String, rightTable: String, cols: Seq[String]): String = {
var selectsql = ""
for (i <- (0 to cols.size-1)) {
selectsql = selectsql + s" if (${leftTable}.${cols(0)} is null, ${rightTable}.${cols(i)}, ${leftTable}.${cols(i)}) as ${cols(i)} ,"
}
"select " + selectsql.dropRight(1) + s" from ${leftTable} full join ${rightTable} on ${leftTable}.${cols(0)} = ${rightTable}.${cols(0)}"
}
/**
* Collect min/max statistics for candidate cols.
* support all col types.
*
* @param df dataFrame holds read files.
* @param cols candidate cols to collect statistics.
* @return
*/
def getMinMaxValueSpark(df: DataFrame, cols: Seq[String]): DataFrame = {
val sqlContext = df.sparkSession.sqlContext
import sqlContext.implicits._
val values = cols.flatMap(c => Seq( min(col(c)).as(c + "_minValue"), max(col(c)).as(c + "_maxValue"), count(c).as(c + "_noNullCount")))
val valueCounts = count("*").as("totalNum")
val projectValues = Seq(col("file")) ++ cols.flatMap(c =>
Seq(col(c + "_minValue"), col(c + "_maxValue"), expr(s"totalNum - ${c + "_noNullCount"}").as(c + "_num_nulls")))
val result = df.select(input_file_name() as "file", col("*"))
.groupBy($"file")
.agg(valueCounts, values: _*).select(projectValues:_*)
result
}
}

View File

@@ -0,0 +1,526 @@
/*
* 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.execution
import java.util
import org.apache.hudi.config.HoodieClusteringConfig
import org.apache.spark.rdd.{PartitionPruningRDD, RDD}
import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, BoundReference, SortOrder, UnsafeProjection, UnsafeRow}
import org.apache.hudi.optimize.ZOrderingUtil
import org.apache.spark.sql.{DataFrame, Row}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering
import org.apache.spark.sql.types._
import org.apache.spark.util.MutablePair
import org.apache.spark.util.random.SamplingUtils
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.reflect.{ClassTag, classTag}
import scala.util.hashing.byteswap32
class RangeSample[K: ClassTag, V](
zEncodeNum: Int,
rdd: RDD[_ <: Product2[K, V]],
private var ascend: Boolean = true,
val samplePointsPerPartitionHint: Int = 20) extends Serializable {
// We allow zEncodeNum = 0, which happens when sorting an empty RDD under the default settings.
require(zEncodeNum >= 0, s"Number of zEncodeNum cannot be negative but found $zEncodeNum.")
require(samplePointsPerPartitionHint > 0,
s"Sample points per partition must be greater than 0 but found $samplePointsPerPartitionHint")
def getRangeBounds(): ArrayBuffer[(K, Float)] = {
if (zEncodeNum <= 1) {
ArrayBuffer.empty[(K, Float)]
} else {
// This is the sample size we need to have roughly balanced output partitions, capped at 1M.
// Cast to double to avoid overflowing ints or longs
val sampleSize = math.min(samplePointsPerPartitionHint.toDouble * zEncodeNum, 1e6)
// Assume the input partitions are roughly balanced and over-sample a little bit.
val sampleSizePerPartition = math.ceil(3.0 * sampleSize / rdd.partitions.length).toInt
val (numItems, sketched) = sketch(rdd.map(_._1), sampleSizePerPartition)
if (numItems == 0L) {
ArrayBuffer.empty[(K, Float)]
} else {
// If a partition contains much more than the average number of items, we re-sample from it
// to ensure that enough items are collected from that partition.
val fraction = math.min(sampleSize / math.max(numItems, 1L), 1.0)
val candidates = ArrayBuffer.empty[(K, Float)]
val imbalancedPartitions = mutable.Set.empty[Int]
sketched.foreach { case (idx, n, sample) =>
if (fraction * n > sampleSizePerPartition) {
imbalancedPartitions += idx
} else {
// The weight is 1 over the sampling probability.
val weight = (n.toDouble / sample.length).toFloat
for (key <- sample) {
candidates += ((key, weight))
}
}
}
if (imbalancedPartitions.nonEmpty) {
// Re-sample imbalanced partitions with the desired sampling probability.
val imbalanced = new PartitionPruningRDD(rdd.map(_._1), imbalancedPartitions.contains)
val seed = byteswap32(-rdd.id - 1)
val reSampled = imbalanced.sample(withReplacement = false, fraction, seed).collect()
val weight = (1.0 / fraction).toFloat
candidates ++= reSampled.map(x => (x, weight))
}
candidates
}
}
}
/**
* Determines the bounds for range partitioning from candidates with weights indicating how many
* items each represents. Usually this is 1 over the probability used to sample this candidate.
*
* @param candidates unordered candidates with weights
* @param partitions number of partitions
* @return selected bounds
*/
def determineBound[K : Ordering : ClassTag](
candidates: ArrayBuffer[(K, Float)],
partitions: Int, ordering: Ordering[K]): Array[K] = {
val ordered = candidates.sortBy(_._1)(ordering)
val numCandidates = ordered.size
val sumWeights = ordered.map(_._2.toDouble).sum
val step = sumWeights / partitions
var cumWeight = 0.0
var target = step
val bounds = ArrayBuffer.empty[K]
var i = 0
var j = 0
var previousBound = Option.empty[K]
while ((i < numCandidates) && (j < partitions - 1)) {
val (key, weight) = ordered(i)
cumWeight += weight
if (cumWeight >= target) {
// Skip duplicate values.
if (previousBound.isEmpty || ordering.gt(key, previousBound.get)) {
bounds += key
target += step
j += 1
previousBound = Some(key)
}
}
i += 1
}
bounds.toArray
}
def determineRowBounds[K : Ordering : ClassTag](
candidates: ArrayBuffer[(K, Float)],
partitions: Int, orderings: Seq[Ordering[K]],
attributes: Seq[Attribute]): Array[Array[UnsafeRow]] = {
orderings.zipWithIndex.map { case (ordering, index) =>
val ordered = candidates.sortBy(_._1)(ordering)
val numCandidates = ordered.size
val sumWeights = ordered.map(_._2.toDouble).sum
val step = sumWeights / partitions
var cumWeight = 0.0
var target = step
val bounds = ArrayBuffer.empty[K]
var i = 0
var j = 0
var previousBound = Option.empty[K]
while ((i < numCandidates) && (j < partitions - 1)) {
val (key, weight) = ordered(i)
cumWeight += weight
if (cumWeight >= target) {
// Skip duplicate values.
if (previousBound.isEmpty || ordering.gt(key, previousBound.get)) {
bounds += key
target += step
j += 1
previousBound = Some(key)
}
}
i += 1
}
// build project
val project = UnsafeProjection.create(Seq(attributes(index)), attributes)
bounds.map { bound =>
val row = bound.asInstanceOf[UnsafeRow]
project(row).copy()
}.toArray
}.toArray
}
/**
* Sketches the input RDD via reservoir sampling on each partition.
*
* @param rdd the input RDD to sketch
* @param sampleSizePerPartition max sample size per partition
* @return (total number of items, an array of (partitionId, number of items, sample))
*/
def sketch[K: ClassTag](
rdd: RDD[K],
sampleSizePerPartition: Int): (Long, Array[(Int, Long, Array[K])]) = {
val shift = rdd.id
// val classTagK = classTag[K] // to avoid serializing the entire partitioner object
val sketched = rdd.mapPartitionsWithIndex { (idx, iter) =>
val seed = byteswap32(idx ^ (shift << 16))
val (sample, n) = SamplingUtils.reservoirSampleAndCount(
iter, sampleSizePerPartition, seed)
Iterator((idx, n, sample))
}.collect()
val numItems = sketched.map(_._2).sum
(numItems, sketched)
}
}
class RawDecisionBound[K : Ordering : ClassTag](ordering: Ordering[K]) extends Serializable {
private var binarySearch: ((Array[K], K) => Int) = {
// For primitive keys, we can use the natural ordering. Otherwise, use the Ordering comparator.
classTag[K] match {
case ClassTag.Float =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Float]], x.asInstanceOf[Float])
case ClassTag.Double =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Double]], x.asInstanceOf[Double])
case ClassTag.Byte =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Byte]], x.asInstanceOf[Byte])
case ClassTag.Char =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Char]], x.asInstanceOf[Char])
case ClassTag.Short =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Short]], x.asInstanceOf[Short])
case ClassTag.Int =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Int]], x.asInstanceOf[Int])
case ClassTag.Long =>
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Long]], x.asInstanceOf[Long])
case _ =>
val comparator = ordering.asInstanceOf[java.util.Comparator[Any]]
(l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[AnyRef]], x, comparator)
}
}
def getBound(key: Any, candidateBounds: Array[K]): Int = {
val k = key.asInstanceOf[K]
var bound = 0
if (candidateBounds.length <= 128) {
while(bound < candidateBounds.length && ordering.gt(k, candidateBounds(bound))) {
bound += 1
}
} else {
bound = binarySearch(candidateBounds, k)
if (bound < 0 ) {
bound = -bound - 1
}
if (bound > candidateBounds.length) {
bound = candidateBounds.length
}
}
bound
}
}
case class ZorderingBinarySort(b: Array[Byte]) extends Ordered[ZorderingBinarySort] with Serializable {
override def compare(that: ZorderingBinarySort): Int = {
val len = this.b.length
ZOrderingUtil.compareTo(this.b, 0, len, that.b, 0, len)
}
}
object RangeSampleSort {
/**
* create z-order DataFrame by sample
* support all col types
*/
def sortDataFrameBySampleSupportAllTypes(df: DataFrame, zCols: Seq[String], fileNum: Int): DataFrame = {
val spark = df.sparkSession
val internalRdd = df.queryExecution.toRdd
val schema = df.schema
val outputAttributes = df.queryExecution.analyzed.output
val sortingExpressions = outputAttributes.filter(p => zCols.contains(p.name))
if (sortingExpressions.length == 0 || sortingExpressions.length != zCols.size) {
df
} else {
val zOrderBounds = df.sparkSession.sessionState.conf.getConfString(
HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.key,
HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.defaultValue.toString).toInt
val sampleRdd = internalRdd.mapPartitionsInternal { iter =>
val projection = UnsafeProjection.create(sortingExpressions, outputAttributes)
val mutablePair = new MutablePair[InternalRow, Null]()
// Internally, RangePartitioner runs a job on the RDD that samples keys to compute
// partition bounds. To get accurate samples, we need to copy the mutable keys.
iter.map(row => mutablePair.update(projection(row).copy(), null))
}
val orderings = sortingExpressions.map(SortOrder(_, Ascending)).zipWithIndex.map { case (ord, i) =>
ord.copy(child = BoundReference(i, ord.dataType, ord.nullable))
}
val lazyGeneratedOrderings = orderings.map(ord => new LazilyGeneratedOrdering(Seq(ord)))
val sample = new RangeSample(zOrderBounds, sampleRdd)
val rangeBounds = sample.getRangeBounds()
implicit val ordering1 = lazyGeneratedOrderings(0)
val sampleBounds = sample.determineRowBounds(rangeBounds, math.min(zOrderBounds, rangeBounds.length), lazyGeneratedOrderings, sortingExpressions)
val origin_orderings = sortingExpressions.map(SortOrder(_, Ascending)).map { ord =>
ord.copy(child = BoundReference(0, ord.dataType, ord.nullable))
}
val origin_lazyGeneratedOrderings = origin_orderings.map(ord => new LazilyGeneratedOrdering(Seq(ord)))
// expand bounds.
// maybe it's better to use the value of "spark.zorder.bounds.number" as maxLength,
// however this will lead to extra time costs when all zorder cols distinct count values are less then "spark.zorder.bounds.number"
val maxLength = sampleBounds.map(_.length).max
val expandSampleBoundsWithFactor = sampleBounds.map { bound =>
val fillFactor = maxLength / bound.size.toDouble
(bound, fillFactor)
}
val boundBroadCast = spark.sparkContext.broadcast(expandSampleBoundsWithFactor)
val indexRdd = internalRdd.mapPartitionsInternal { iter =>
val boundsWithFactor = boundBroadCast.value
import java.util.concurrent.ThreadLocalRandom
val threadLocalRandom = ThreadLocalRandom.current
val maxBoundNum = boundsWithFactor.map(_._1.length).max
val origin_Projections = sortingExpressions.map { se =>
UnsafeProjection.create(Seq(se), outputAttributes)
}
iter.map { unsafeRow =>
val interleaveValues = origin_Projections.zip(origin_lazyGeneratedOrderings).zipWithIndex.map { case ((rowProject, lazyOrdering), index) =>
val row = rowProject(unsafeRow)
val decisionBound = new RawDecisionBound(lazyOrdering)
if (row.isNullAt(0)) {
maxBoundNum + 1
} else {
val (bound, factor) = boundsWithFactor(index)
if (factor > 1) {
val currentRank = decisionBound.getBound(row, bound.asInstanceOf[Array[InternalRow]])
currentRank*factor.toInt + threadLocalRandom.nextInt(factor.toInt)
} else {
decisionBound.getBound(row, bound.asInstanceOf[Array[InternalRow]])
}
}
}.toArray.map(ZOrderingUtil.intTo8Byte(_))
val zValues = ZOrderingUtil.interleaving(interleaveValues, 8)
val mutablePair = new MutablePair[InternalRow, Array[Byte]]()
mutablePair.update(unsafeRow, zValues)
}
}.sortBy(x => ZorderingBinarySort(x._2), numPartitions = fileNum).map(_._1)
spark.internalCreateDataFrame(indexRdd, schema)
}
}
/**
* 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 sortDataFrameBySample(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 sortDataFrameBySampleSupportAllTypes(df, zCols, fieldNum)
}
val rawRdd = df.rdd
val sampleRdd = rawRdd.map { row =>
val values = zFields.map { case (index, field) =>
field.dataType match {
case LongType =>
if (row.isNullAt(index)) Long.MaxValue else row.getLong(index)
case DoubleType =>
if (row.isNullAt(index)) Long.MaxValue else java.lang.Double.doubleToLongBits(row.getDouble(index))
case IntegerType =>
if (row.isNullAt(index)) Long.MaxValue else row.getInt(index).toLong
case FloatType =>
if (row.isNullAt(index)) Long.MaxValue else java.lang.Double.doubleToLongBits(row.getFloat(index).toDouble)
case StringType =>
if (row.isNullAt(index)) "" else row.getString(index)
case DateType =>
if (row.isNullAt(index)) Long.MaxValue else row.getDate(index).getTime
case TimestampType =>
if (row.isNullAt(index)) Long.MaxValue else row.getTimestamp(index).getTime
case ByteType =>
if (row.isNullAt(index)) Long.MaxValue else row.getByte(index).toLong
case ShortType =>
if (row.isNullAt(index)) Long.MaxValue else row.getShort(index).toLong
case d: DecimalType =>
if (row.isNullAt(index)) Long.MaxValue else row.getDecimal(index).longValue()
case _ =>
null
}
}.filter(v => v != null).toArray
(values, null)
}
val zOrderBounds = df.sparkSession.sessionState.conf.getConfString(
HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.key,
HoodieClusteringConfig.LAYOUT_OPTIMIZE_BUILD_CURVE_SAMPLE_SIZE.defaultValue.toString).toInt
val sample = new RangeSample(zOrderBounds, sampleRdd)
val rangeBounds = sample.getRangeBounds()
val sampleBounds = {
val candidateColNumber = rangeBounds.head._1.length
(0 to candidateColNumber - 1).map { i =>
val colRangeBound = rangeBounds.map(x => (x._1(i), x._2))
if (colRangeBound.head._1.isInstanceOf[String]) {
sample.determineBound(colRangeBound.asInstanceOf[ArrayBuffer[(String, Float)]], math.min(zOrderBounds, rangeBounds.length), Ordering[String])
} else {
sample.determineBound(colRangeBound.asInstanceOf[ArrayBuffer[(Long, Float)]], math.min(zOrderBounds, rangeBounds.length), Ordering[Long])
}
}
}
// expand bounds.
// maybe it's better to use the value of "spark.zorder.bounds.number" as maxLength,
// however this will lead to extra time costs when all zorder cols distinct count values are less then "spark.zorder.bounds.number"
val maxLength = sampleBounds.map(_.length).max
val expandSampleBoundsWithFactor = sampleBounds.map { bound =>
val fillFactor = maxLength / bound.size
val newBound = new Array[Double](bound.length * fillFactor)
if (bound.isInstanceOf[Array[Long]] && fillFactor > 1) {
val longBound = bound.asInstanceOf[Array[Long]]
for (i <- 0 to bound.length - 1) {
for (j <- 0 to fillFactor - 1) {
// sample factor shoud not be too large, so it's ok to use 1 / fillfactor as slice
newBound(j + i*(fillFactor)) = longBound(i) + (j + 1) * (1 / fillFactor.toDouble)
}
}
(newBound, fillFactor)
} else {
(bound, 0)
}
}
val boundBroadCast = spark.sparkContext.broadcast(expandSampleBoundsWithFactor)
val indexRdd = rawRdd.mapPartitions { iter =>
val expandBoundsWithFactor = boundBroadCast.value
val maxBoundNum = expandBoundsWithFactor.map(_._1.length).max
val longDecisionBound = new RawDecisionBound(Ordering[Long])
val doubleDecisionBound = new RawDecisionBound(Ordering[Double])
val stringDecisionBound = new RawDecisionBound(Ordering[String])
import java.util.concurrent.ThreadLocalRandom
val threadLocalRandom = ThreadLocalRandom.current
def getRank(rawIndex: Int, value: Long, isNull: Boolean): Int = {
val (expandBound, factor) = expandBoundsWithFactor(rawIndex)
if (isNull) {
expandBound.length + 1
} else {
if (factor > 1) {
doubleDecisionBound.getBound(value + (threadLocalRandom.nextInt(factor) + 1)*(1 / factor.toDouble), expandBound.asInstanceOf[Array[Double]])
} else {
longDecisionBound.getBound(value, expandBound.asInstanceOf[Array[Long]])
}
}
}
iter.map { row =>
val values = zFields.zipWithIndex.map { case ((index, field), rawIndex) =>
field.dataType match {
case LongType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getLong(index), isNull)
case DoubleType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else java.lang.Double.doubleToLongBits(row.getDouble(index)), isNull)
case IntegerType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getInt(index).toLong, isNull)
case FloatType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else java.lang.Double.doubleToLongBits(row.getFloat(index).toDouble), isNull)
case StringType =>
val factor = maxBoundNum.toDouble / expandBoundsWithFactor(rawIndex)._1.length
if (row.isNullAt(index)) {
maxBoundNum + 1
} else {
val currentRank = stringDecisionBound.getBound(row.getString(index), expandBoundsWithFactor(rawIndex)._1.asInstanceOf[Array[String]])
if (factor > 1) {
(currentRank*factor).toInt + threadLocalRandom.nextInt(factor.toInt)
} else {
currentRank
}
}
case DateType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getDate(index).getTime, isNull)
case TimestampType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getTimestamp(index).getTime, isNull)
case ByteType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getByte(index).toLong, isNull)
case ShortType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getShort(index).toLong, isNull)
case d: DecimalType =>
val isNull = row.isNullAt(index)
getRank(rawIndex, if (isNull) 0 else row.getDecimal(index).longValue(), isNull)
case _ =>
-1
}
}.filter(v => v != -1).map(ZOrderingUtil.intTo8Byte(_)).toArray
val zValues = ZOrderingUtil.interleaving(values, 8)
Row.fromSeq(row.toSeq ++ Seq(zValues))
}
}.sortBy(x => ZorderingBinarySort(x.getAs[Array[Byte]](fieldNum)), numPartitions = fileNum)
val newDF = df.sparkSession.createDataFrame(indexRdd, StructType(
df.schema.fields ++ Seq(
StructField(s"zindex",
BinaryType, false))
))
newDF.drop("zindex")
}
}
}