[HUDI-2814] Addressing issues w/ Z-order Layout Optimization (#4060)
* `ZCurveOptimizeHelper` > `ZOrderingIndexHelper`; Moved Z-index helper under `hudi.index.zorder` package * Tidying up `ZOrderingIndexHelper` * Fixing compilation * Fixed index new/original table merging sequence to always prefer values from new index; Cleaned up `HoodieSparkUtils` * Added test for `mergeIndexSql` * Abstracted Z-index name composition w/in `ZOrderingIndexHelper`; * Fixed `DataSkippingUtils` to interrupt prunning in case data filter contains non-indexed column reference * Properly handle exceptions origination during pruning in `HoodieFileIndex` * Make sure no errors are logged upon encountering `AnalysisException` * Cleaned up Z-index updating sequence; Tidying up comments, java-docs; * Fixed Z-index to properly handle changes of the list of clustered columns * Tidying up * `lint` * Suppressing `JavaDocStyle` first sentence check * Fixed compilation * Fixing incorrect `DecimalType` conversion * Refactored test `TestTableLayoutOptimization` - Added Z-index table composition test (against fixtures) - Separated out GC test; Tidying up * Fixed tests re-shuffling column order for Z-Index table `DataFrame` to align w/ the one by one loaded from JSON * Scaffolded `DataTypeUtils` to do basic checks of Spark types; Added proper compatibility checking b/w old/new index-tables * Added test for Z-index tables merging * Fixed import being shaded by creating internal `hudi.util` package * Fixed packaging for `TestOptimizeTable` * Revised `updateMetadataIndex` seq to provide Z-index updating process w/ source table schema * Make sure existing Z-index table schema is sync'd to source table's one * Fixed shaded refs * Fixed tests * Fixed type conversion of Parquet provided metadata values into Spark expected schemas * Fixed `composeIndexSchema` utility to propose proper schema * Added more tests for Z-index: - Checking that Z-index table is built correctly - Checking that Z-index tables are merged correctly (during update) * Fixing source table * Fixing tests to read from Parquet w/ proper schema * Refactored `ParquetUtils` utility reading stats from Parquet footers * Fixed incorrect handling of Decimals extracted from Parquet footers * Worked around issues in javac failign to compile stream's collection * Fixed handling of `Date` type * Fixed handling of `DateType` to be parsed as `LocalDate` * Updated fixture; Make sure test loads Z-index fixture using proper schema * Removed superfluous scheme adjusting when reading from Parquet, since Spark is actually able to perfectly restore schema (given Parquet was previously written by Spark as well) * Fixing race-condition in Parquet's `DateStringifier` trying to share `SimpleDataFormat` object which is inherently not thread-safe * Tidying up * Make sure schema is used upon reading to validate input files are in the appropriate format; Tidying up; * Worked around javac (1.8) inability to infer expression type properly * Updated fixtures; Tidying up * Fixing compilation after rebase * Assert clustering have in Z-order layout optimization testing * Tidying up exception messages * XXX * Added test validating Z-index lookup filter correctness * Added more test-cases; Tidying up * Added tests for string expressions * Fixed incorrect Z-index filter lookup translations * Added more test-cases * Added proper handling on complex negations of AND/OR expressions by pushing NOT operator down into inner expressions for appropriate handling * Added `-target:jvm-1.8` for `hudi-spark` module * Adding more tests * Added tests for non-indexed columns * Properly handle non-indexed columns by falling back to a re-write of containing expression as `TrueLiteral` instead * Fixed tests * Removing the parquet test files and disabling corresponding tests Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -396,15 +396,16 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty());
|
||||
finalizeWrite(table, clusteringCommitTime, writeStats);
|
||||
writeTableMetadataForTableServices(table, metadata,clusteringInstant);
|
||||
// try to save statistics info to hudi
|
||||
if (config.isDataSkippingEnabled() && config.isLayoutOptimizationEnabled() && !config.getClusteringSortColumns().isEmpty()) {
|
||||
table.updateStatistics(context, writeStats, clusteringCommitTime, true);
|
||||
// Update outstanding metadata indexes
|
||||
if (config.isLayoutOptimizationEnabled()
|
||||
&& !config.getClusteringSortColumns().isEmpty()) {
|
||||
table.updateMetadataIndexes(context, writeStats, clusteringCommitTime);
|
||||
}
|
||||
LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata);
|
||||
table.getActiveTimeline().transitionReplaceInflightToComplete(
|
||||
HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException e) {
|
||||
} catch (Exception e) {
|
||||
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
|
||||
@@ -33,7 +33,7 @@ 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.hudi.index.zorder.ZOrderingIndexHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
@@ -79,10 +79,10 @@ public class RDDSpatialCurveOptimizationSortPartitioner<T extends HoodieRecordPa
|
||||
|
||||
switch (config.getLayoutOptimizationCurveBuildMethod()) {
|
||||
case DIRECT:
|
||||
zDataFrame = ZCurveOptimizeHelper.createZIndexedDataFrameByMapValue(originDF, config.getClusteringSortColumns(), numOutputGroups);
|
||||
zDataFrame = ZOrderingIndexHelper.createZIndexedDataFrameByMapValue(originDF, config.getClusteringSortColumns(), numOutputGroups);
|
||||
break;
|
||||
case SAMPLE:
|
||||
zDataFrame = ZCurveOptimizeHelper.createZIndexedDataFrameBySample(originDF, config.getClusteringSortColumns(), numOutputGroups);
|
||||
zDataFrame = ZOrderingIndexHelper.createZIndexedDataFrameBySample(originDF, config.getClusteringSortColumns(), numOutputGroups);
|
||||
break;
|
||||
default:
|
||||
throw new HoodieException("Not a valid build curve method for doWriteOperation: ");
|
||||
|
||||
@@ -0,0 +1,619 @@
|
||||
/*
|
||||
* 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.index.zorder;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.optimize.ZOrderingUtil;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.io.api.Binary;
|
||||
import org.apache.spark.SparkContext;
|
||||
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.StructType$;
|
||||
import org.apache.spark.sql.types.TimestampType;
|
||||
import org.apache.spark.util.SerializableConfiguration;
|
||||
import scala.collection.JavaConversions;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.IOException;
|
||||
import java.math.BigDecimal;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
import static org.apache.hudi.util.DataTypeUtils.areCompatible;
|
||||
|
||||
public class ZOrderingIndexHelper {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(ZOrderingIndexHelper.class);
|
||||
|
||||
private static final String SPARK_JOB_DESCRIPTION = "spark.job.description";
|
||||
|
||||
private static final String Z_INDEX_FILE_COLUMN_NAME = "file";
|
||||
|
||||
private static final String Z_INDEX_MIN_VALUE_STAT_NAME = "minValue";
|
||||
private static final String Z_INDEX_MAX_VALUE_STAT_NAME = "maxValue";
|
||||
private static final String Z_INDEX_NUM_NULLS_STAT_NAME = "num_nulls";
|
||||
|
||||
public static String getMinColumnNameFor(String colName) {
|
||||
return composeZIndexColName(colName, Z_INDEX_MIN_VALUE_STAT_NAME);
|
||||
}
|
||||
|
||||
public static String getMaxColumnNameFor(String colName) {
|
||||
return composeZIndexColName(colName, Z_INDEX_MAX_VALUE_STAT_NAME);
|
||||
}
|
||||
|
||||
public static String getNumNullsColumnNameFor(String colName) {
|
||||
return composeZIndexColName(colName, Z_INDEX_NUM_NULLS_STAT_NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 from Parquet footers for provided columns and composes Z-index
|
||||
* table in the following format with 3 statistics denominated for each Z-ordered column.
|
||||
* For ex, if original table contained Z-ordered column {@code A}:
|
||||
*
|
||||
* <pre>
|
||||
* +---------------------------+------------+------------+-------------+
|
||||
* | file | A_minValue | A_maxValue | A_num_nulls |
|
||||
* +---------------------------+------------+------------+-------------+
|
||||
* | one_base_file.parquet | 1 | 10 | 0 |
|
||||
* | another_base_file.parquet | -10 | 0 | 5 |
|
||||
* +---------------------------+------------+------------+-------------+
|
||||
* </pre>
|
||||
*
|
||||
* NOTE: Currently {@link TimestampType} is not supported, since Parquet writer
|
||||
* does not support statistics for it.
|
||||
*
|
||||
* TODO leverage metadata table after RFC-27 lands
|
||||
* @VisibleForTesting
|
||||
*
|
||||
* @param sparkSession encompassing Spark session
|
||||
* @param baseFilesPaths list of base-files paths to be sourced for Z-index
|
||||
* @param zorderedColumnSchemas target Z-ordered columns
|
||||
* @return Spark's {@link Dataset} holding an index table
|
||||
*/
|
||||
@Nonnull
|
||||
public static Dataset<Row> buildZIndexTableFor(
|
||||
@Nonnull SparkSession sparkSession,
|
||||
@Nonnull List<String> baseFilesPaths,
|
||||
@Nonnull List<StructField> zorderedColumnSchemas
|
||||
) {
|
||||
SparkContext sc = sparkSession.sparkContext();
|
||||
JavaSparkContext jsc = new JavaSparkContext(sc);
|
||||
|
||||
SerializableConfiguration serializableConfiguration = new SerializableConfiguration(sc.hadoopConfiguration());
|
||||
int numParallelism = (baseFilesPaths.size() / 3 + 1);
|
||||
List<HoodieColumnRangeMetadata<Comparable>> colMinMaxInfos;
|
||||
String previousJobDescription = sc.getLocalProperty(SPARK_JOB_DESCRIPTION);
|
||||
try {
|
||||
jsc.setJobDescription("Listing parquet column statistics");
|
||||
colMinMaxInfos =
|
||||
jsc.parallelize(baseFilesPaths, numParallelism)
|
||||
.mapPartitions(paths -> {
|
||||
ParquetUtils utils = (ParquetUtils) BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
|
||||
Iterable<String> iterable = () -> paths;
|
||||
return StreamSupport.stream(iterable.spliterator(), false)
|
||||
.flatMap(path ->
|
||||
utils.readRangeFromParquetMetadata(
|
||||
serializableConfiguration.value(),
|
||||
new Path(path),
|
||||
zorderedColumnSchemas.stream()
|
||||
.map(StructField::name)
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
.stream()
|
||||
)
|
||||
.iterator();
|
||||
})
|
||||
.collect();
|
||||
} finally {
|
||||
jsc.setJobDescription(previousJobDescription);
|
||||
}
|
||||
|
||||
// Group column's metadata by file-paths of the files it belongs to
|
||||
Map<String, List<HoodieColumnRangeMetadata<Comparable>>> filePathToColumnMetadataMap =
|
||||
colMinMaxInfos.stream()
|
||||
.collect(Collectors.groupingBy(HoodieColumnRangeMetadata::getFilePath));
|
||||
|
||||
JavaRDD<Row> allMetaDataRDD =
|
||||
jsc.parallelize(new ArrayList<>(filePathToColumnMetadataMap.values()), 1)
|
||||
.map(fileColumnsMetadata -> {
|
||||
int colSize = fileColumnsMetadata.size();
|
||||
if (colSize == 0) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String filePath = fileColumnsMetadata.get(0).getFilePath();
|
||||
|
||||
List<Object> indexRow = new ArrayList<>();
|
||||
|
||||
// First columns of the Z-index's row is target file-path
|
||||
indexRow.add(filePath);
|
||||
|
||||
// For each column
|
||||
zorderedColumnSchemas.forEach(colSchema -> {
|
||||
String colName = colSchema.name();
|
||||
|
||||
HoodieColumnRangeMetadata<Comparable> colMetadata =
|
||||
fileColumnsMetadata.stream()
|
||||
.filter(s -> s.getColumnName().trim().equalsIgnoreCase(colName))
|
||||
.findFirst()
|
||||
.orElse(null);
|
||||
|
||||
DataType colType = colSchema.dataType();
|
||||
if (colMetadata == null || colType == null) {
|
||||
throw new HoodieException(String.format("Cannot collect min/max statistics for column (%s)", colSchema));
|
||||
}
|
||||
|
||||
Pair<Object, Object> minMaxValue = fetchMinMaxValues(colType, colMetadata);
|
||||
|
||||
indexRow.add(minMaxValue.getLeft()); // min
|
||||
indexRow.add(minMaxValue.getRight()); // max
|
||||
indexRow.add(colMetadata.getNumNulls());
|
||||
});
|
||||
|
||||
return Row$.MODULE$.apply(JavaConversions.asScalaBuffer(indexRow));
|
||||
})
|
||||
.filter(Objects::nonNull);
|
||||
|
||||
StructType indexSchema = composeIndexSchema(zorderedColumnSchemas);
|
||||
|
||||
return sparkSession.createDataFrame(allMetaDataRDD, indexSchema);
|
||||
}
|
||||
|
||||
/**
|
||||
* <p/>
|
||||
* Updates state of the Z-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>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 commitTime current operation commit instant
|
||||
* @param completedCommits all previously completed commit instants
|
||||
*/
|
||||
public static void updateZIndexFor(
|
||||
@Nonnull SparkSession sparkSession,
|
||||
@Nonnull StructType sourceTableSchema,
|
||||
@Nonnull List<String> sourceBaseFiles,
|
||||
@Nonnull List<String> zorderedCols,
|
||||
@Nonnull String zindexFolderPath,
|
||||
@Nonnull String commitTime,
|
||||
@Nonnull List<String> completedCommits
|
||||
) {
|
||||
FileSystem fs = FSUtils.getFs(zindexFolderPath, sparkSession.sparkContext().hadoopConfiguration());
|
||||
|
||||
// Compose new Z-index table for the given source base files
|
||||
Dataset<Row> newZIndexDf =
|
||||
buildZIndexTableFor(
|
||||
sparkSession,
|
||||
sourceBaseFiles,
|
||||
zorderedCols.stream()
|
||||
.map(col -> sourceTableSchema.fields()[sourceTableSchema.fieldIndex(col)])
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
|
||||
try {
|
||||
//
|
||||
// Z-Index has the following folder structure:
|
||||
//
|
||||
// .hoodie/
|
||||
// ├── .zindex/
|
||||
// │ ├── <instant>/
|
||||
// │ │ ├── <part-...>.parquet
|
||||
// │ │ └── ...
|
||||
//
|
||||
// If index is currently empty (no persisted tables), we simply create one
|
||||
// using clustering operation's commit instance as it's name
|
||||
Path newIndexTablePath = new Path(zindexFolderPath, commitTime);
|
||||
|
||||
if (!fs.exists(new Path(zindexFolderPath))) {
|
||||
newZIndexDf.repartition(1)
|
||||
.write()
|
||||
.format("parquet")
|
||||
.mode("overwrite")
|
||||
.save(newIndexTablePath.toString());
|
||||
return;
|
||||
}
|
||||
|
||||
// Filter in all index tables (w/in {@code .zindex} folder)
|
||||
List<String> allIndexTables =
|
||||
Arrays.stream(
|
||||
fs.listStatus(new Path(zindexFolderPath))
|
||||
)
|
||||
.filter(FileStatus::isDirectory)
|
||||
.map(f -> f.getPath().getName())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Compile list of valid index tables that were produced as part
|
||||
// of previously successfully committed iterations
|
||||
List<String> validIndexTables =
|
||||
allIndexTables.stream()
|
||||
.filter(completedCommits::contains)
|
||||
.sorted()
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<String> tablesToCleanup =
|
||||
allIndexTables.stream()
|
||||
.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
|
||||
// with the most recent one that were successfully persisted previously
|
||||
if (validIndexTables.isEmpty()) {
|
||||
finalZIndexDf = newZIndexDf;
|
||||
} 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 =
|
||||
tryMergeMostRecentIndexTableInto(
|
||||
sparkSession,
|
||||
newZIndexDf,
|
||||
// Load current most recent Z-index table
|
||||
sparkSession.read().load(
|
||||
new Path(zindexFolderPath, 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());
|
||||
|
||||
// Clean up residual Z-index tables that have might have been dangling since
|
||||
// previous iterations (due to intermittent failures during previous clean up)
|
||||
tablesToCleanup.forEach(f -> {
|
||||
try {
|
||||
fs.delete(new Path(zindexFolderPath, f), true);
|
||||
} catch (IOException ie) {
|
||||
// NOTE: Exception is deliberately swallowed to not affect overall clustering operation,
|
||||
// since failing Z-index table will be attempted to be cleaned up upon subsequent
|
||||
// clustering iteration
|
||||
LOG.warn(String.format("Failed to cleanup residual Z-index table: %s", f), ie);
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to build new Z-index table", e);
|
||||
throw new HoodieException("Failed to build new Z-index table", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Nonnull
|
||||
private static Dataset<Row> tryMergeMostRecentIndexTableInto(
|
||||
@Nonnull SparkSession sparkSession,
|
||||
@Nonnull Dataset<Row> newIndexTableDf,
|
||||
@Nonnull Dataset<Row> existingIndexTableDf
|
||||
) {
|
||||
// NOTE: If new Z-index table schema is incompatible with that one of existing table
|
||||
// that is most likely due to changing settings of list of Z-ordered columns, that
|
||||
// occurred since last index table have been persisted.
|
||||
//
|
||||
// In that case, we simply drop existing index table and just persist the new one;
|
||||
//
|
||||
// Also note that we're checking compatibility of _old_ index-table with new one and that
|
||||
// COMPATIBILITY OPERATION DOES NOT COMMUTE (ie if A is compatible w/ B,
|
||||
// B might not necessarily be compatible w/ A)
|
||||
if (!areCompatible(existingIndexTableDf.schema(), newIndexTableDf.schema())) {
|
||||
return newIndexTableDf;
|
||||
}
|
||||
|
||||
String randomSuffix = UUID.randomUUID().toString().replace("-", "");
|
||||
|
||||
String existingIndexTempTableName = "existingIndexTable_" + randomSuffix;
|
||||
String newIndexTempTableName = "newIndexTable_" + randomSuffix;
|
||||
|
||||
existingIndexTableDf.registerTempTable(existingIndexTempTableName);
|
||||
newIndexTableDf.registerTempTable(newIndexTempTableName);
|
||||
|
||||
List<String> newTableColumns = Arrays.asList(newIndexTableDf.schema().fieldNames());
|
||||
|
||||
// Create merged table by doing full-out join
|
||||
return sparkSession.sql(createIndexMergeSql(existingIndexTempTableName, newIndexTempTableName, newTableColumns));
|
||||
}
|
||||
|
||||
/**
|
||||
* @VisibleForTesting
|
||||
*/
|
||||
@Nonnull
|
||||
public static StructType composeIndexSchema(@Nonnull List<StructField> zorderedColumnsSchemas) {
|
||||
List<StructField> schema = new ArrayList<>();
|
||||
schema.add(new StructField(Z_INDEX_FILE_COLUMN_NAME, StringType$.MODULE$, true, Metadata.empty()));
|
||||
zorderedColumnsSchemas.forEach(colSchema -> {
|
||||
schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_MIN_VALUE_STAT_NAME, colSchema.dataType()));
|
||||
schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_MAX_VALUE_STAT_NAME, colSchema.dataType()));
|
||||
schema.add(composeColumnStatStructType(colSchema.name(), Z_INDEX_NUM_NULLS_STAT_NAME, LongType$.MODULE$));
|
||||
});
|
||||
return StructType$.MODULE$.apply(schema);
|
||||
}
|
||||
|
||||
private static StructField composeColumnStatStructType(String col, String statName, DataType dataType) {
|
||||
return new StructField(composeZIndexColName(col, statName), dataType, true, Metadata.empty());
|
||||
}
|
||||
|
||||
@Nullable
|
||||
private static String mapToSourceTableColumnName(StructField fieldStruct) {
|
||||
String name = fieldStruct.name();
|
||||
int maxStatSuffixIdx = name.lastIndexOf(String.format("_%s", Z_INDEX_MAX_VALUE_STAT_NAME));
|
||||
if (maxStatSuffixIdx != -1) {
|
||||
return name.substring(0, maxStatSuffixIdx);
|
||||
}
|
||||
|
||||
int minStatSuffixIdx = name.lastIndexOf(String.format("_%s", Z_INDEX_MIN_VALUE_STAT_NAME));
|
||||
if (minStatSuffixIdx != -1) {
|
||||
return name.substring(0, minStatSuffixIdx);
|
||||
}
|
||||
|
||||
int numNullsSuffixIdx = name.lastIndexOf(String.format("_%s", Z_INDEX_NUM_NULLS_STAT_NAME));
|
||||
if (numNullsSuffixIdx != -1) {
|
||||
return name.substring(0, numNullsSuffixIdx);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
private static String composeZIndexColName(String col, String statName) {
|
||||
// TODO add escaping for
|
||||
return String.format("%s_%s", col, statName);
|
||||
}
|
||||
|
||||
private static Pair<Object, Object>
|
||||
fetchMinMaxValues(
|
||||
@Nonnull DataType colType,
|
||||
@Nonnull HoodieColumnRangeMetadata<Comparable> colMetadata) {
|
||||
if (colType instanceof IntegerType) {
|
||||
return Pair.of(
|
||||
new Integer(colMetadata.getMinValue().toString()),
|
||||
new Integer(colMetadata.getMaxValue().toString())
|
||||
);
|
||||
} else if (colType instanceof DoubleType) {
|
||||
return Pair.of(
|
||||
new Double(colMetadata.getMinValue().toString()),
|
||||
new Double(colMetadata.getMaxValue().toString())
|
||||
);
|
||||
} else if (colType instanceof StringType) {
|
||||
return Pair.of(
|
||||
new String(((Binary) colMetadata.getMinValue()).getBytes()),
|
||||
new String(((Binary) colMetadata.getMaxValue()).getBytes())
|
||||
);
|
||||
} else if (colType instanceof DecimalType) {
|
||||
return Pair.of(
|
||||
new BigDecimal(colMetadata.getMinValue().toString()),
|
||||
new BigDecimal(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof DateType) {
|
||||
return Pair.of(
|
||||
java.sql.Date.valueOf(colMetadata.getMinValue().toString()),
|
||||
java.sql.Date.valueOf(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof LongType) {
|
||||
return Pair.of(
|
||||
new Long(colMetadata.getMinValue().toString()),
|
||||
new Long(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof ShortType) {
|
||||
return Pair.of(
|
||||
new Short(colMetadata.getMinValue().toString()),
|
||||
new Short(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof FloatType) {
|
||||
return Pair.of(
|
||||
new Float(colMetadata.getMinValue().toString()),
|
||||
new Float(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof BinaryType) {
|
||||
return Pair.of(
|
||||
((Binary) colMetadata.getMinValue()).getBytes(),
|
||||
((Binary) colMetadata.getMaxValue()).getBytes());
|
||||
} else if (colType instanceof BooleanType) {
|
||||
return Pair.of(
|
||||
Boolean.valueOf(colMetadata.getMinValue().toString()),
|
||||
Boolean.valueOf(colMetadata.getMaxValue().toString()));
|
||||
} else if (colType instanceof ByteType) {
|
||||
return Pair.of(
|
||||
Byte.valueOf(colMetadata.getMinValue().toString()),
|
||||
Byte.valueOf(colMetadata.getMaxValue().toString()));
|
||||
} else {
|
||||
throw new HoodieException(String.format("Not support type: %s", colType));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @VisibleForTesting
|
||||
*/
|
||||
@Nonnull
|
||||
static String createIndexMergeSql(
|
||||
@Nonnull String originalIndexTable,
|
||||
@Nonnull String newIndexTable,
|
||||
@Nonnull List<String> columns
|
||||
) {
|
||||
StringBuilder selectBody = new StringBuilder();
|
||||
|
||||
for (int i = 0; i < columns.size(); ++i) {
|
||||
String col = columns.get(i);
|
||||
String originalTableColumn = String.format("%s.%s", originalIndexTable, col);
|
||||
String newTableColumn = String.format("%s.%s", newIndexTable, col);
|
||||
|
||||
selectBody.append(
|
||||
// NOTE: We prefer values from the new index table, and fallback to the original one only
|
||||
// in case it does not contain statistics for the given file path
|
||||
String.format("if (%s is null, %s, %s) AS %s", newTableColumn, originalTableColumn, newTableColumn, col)
|
||||
);
|
||||
|
||||
if (i < columns.size() - 1) {
|
||||
selectBody.append(", ");
|
||||
}
|
||||
}
|
||||
|
||||
return String.format(
|
||||
"SELECT %s FROM %s FULL JOIN %s ON %s = %s",
|
||||
selectBody,
|
||||
originalIndexTable,
|
||||
newIndexTable,
|
||||
String.format("%s.%s", originalIndexTable, columns.get(0)),
|
||||
String.format("%s.%s", newIndexTable, columns.get(0))
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -18,7 +18,10 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieClusteringPlan;
|
||||
@@ -37,9 +40,11 @@ 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.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
@@ -73,11 +78,12 @@ 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.hudi.index.zorder.ZOrderingIndexHelper;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.collection.JavaConversions;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
@@ -163,29 +169,61 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload>
|
||||
}
|
||||
|
||||
@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);
|
||||
}
|
||||
public void updateMetadataIndexes(@Nonnull HoodieEngineContext context, @Nonnull List<HoodieWriteStat> stats, @Nonnull String instantTime) throws Exception {
|
||||
// Updates Z-ordering Index
|
||||
updateZIndex(context, stats, instantTime);
|
||||
}
|
||||
|
||||
private void updateOptimizeOperationStatistics(HoodieEngineContext context, List<HoodieWriteStat> stats, String instantTime) {
|
||||
String cols = config.getClusteringSortColumns();
|
||||
private void updateZIndex(
|
||||
@Nonnull HoodieEngineContext context,
|
||||
@Nonnull List<HoodieWriteStat> updatedFilesStats,
|
||||
@Nonnull String instantTime
|
||||
) throws Exception {
|
||||
String sortColsList = 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");
|
||||
|
||||
List<String> completedCommits =
|
||||
metaClient.getCommitsTimeline()
|
||||
.filterCompletedInstants()
|
||||
.getInstants()
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
List<String> touchedFiles =
|
||||
updatedFilesStats.stream()
|
||||
.map(s -> new Path(basePath, s.getPath()).toString())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
if (touchedFiles.isEmpty() || StringUtils.isNullOrEmpty(sortColsList) || StringUtils.isNullOrEmpty(indexPath)) {
|
||||
return;
|
||||
}
|
||||
|
||||
LOG.info(String.format("Updating Z-index table (%s)", indexPath));
|
||||
|
||||
List<String> sortCols = Arrays.stream(sortColsList.split(","))
|
||||
.map(String::trim)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
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));
|
||||
|
||||
// Fetch table schema to appropriately construct Z-index schema
|
||||
Schema tableWriteSchema =
|
||||
HoodieAvroUtils.createHoodieWriteSchema(
|
||||
new TableSchemaResolver(metaClient).getTableAvroSchemaWithoutMetadataFields()
|
||||
);
|
||||
|
||||
ZOrderingIndexHelper.updateZIndexFor(
|
||||
sparkEngineContext.getSqlContext().sparkSession(),
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(tableWriteSchema),
|
||||
touchedFiles,
|
||||
sortCols,
|
||||
indexPath,
|
||||
instantTime,
|
||||
completedCommits
|
||||
);
|
||||
|
||||
LOG.info(String.format("Successfully updated Z-index at instant (%s)", instantTime));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -0,0 +1,122 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import org.apache.spark.sql.types.ByteType$;
|
||||
import org.apache.spark.sql.types.DataType;
|
||||
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.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.VarcharType$;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
||||
public class DataTypeUtils {
|
||||
|
||||
private static Map<Class<?>, Set<Class<?>>> sparkPrimitiveTypesCompatibilityMap =
|
||||
new HashMap<Class<?>, Set<Class<?>>>() {{
|
||||
|
||||
// Integral types
|
||||
put(ShortType$.class,
|
||||
newHashSet(ByteType$.class, ShortType$.class));
|
||||
put(IntegerType$.class,
|
||||
newHashSet(ByteType$.class, ShortType$.class, IntegerType$.class));
|
||||
put(LongType$.class,
|
||||
newHashSet(ByteType$.class, ShortType$.class, IntegerType$.class, LongType$.class));
|
||||
|
||||
// Float types
|
||||
put(DoubleType$.class,
|
||||
newHashSet(FloatType$.class, DoubleType$.class));
|
||||
|
||||
// String types
|
||||
put(StringType$.class,
|
||||
newHashSet(VarcharType$.class, StringType$.class));
|
||||
}};
|
||||
|
||||
/**
|
||||
* Validates whether one {@link StructType} is compatible w/ the other one.
|
||||
* Compatibility rules are defined like following: types A and B are considered
|
||||
* compatible iff
|
||||
*
|
||||
* <ol>
|
||||
* <li>A and B are identical</li>
|
||||
* <li>All values comprising A domain are contained w/in B domain (for ex, {@code ShortType}
|
||||
* in this sense is compatible w/ {@code IntegerType})</li>
|
||||
* </ol>
|
||||
*
|
||||
* @param left operand
|
||||
* @param right operand
|
||||
* @return true if {@code left} instance of {@link StructType} is compatible w/ the {@code right}
|
||||
*/
|
||||
public static boolean areCompatible(@Nonnull DataType left, @Nonnull DataType right) {
|
||||
// First, check if types are equal
|
||||
if (Objects.equals(left, right)) {
|
||||
return true;
|
||||
}
|
||||
|
||||
// If not, check whether both are instances of {@code StructType} that
|
||||
// should be matched structurally
|
||||
if (left instanceof StructType && right instanceof StructType) {
|
||||
return areCompatible((StructType) left, (StructType) right);
|
||||
}
|
||||
|
||||
// If not, simply check if those data-types constitute compatibility
|
||||
// relationship outlined above; otherwise return false
|
||||
return sparkPrimitiveTypesCompatibilityMap.getOrDefault(left.getClass(), Collections.emptySet())
|
||||
.contains(right.getClass());
|
||||
}
|
||||
|
||||
private static boolean areCompatible(@Nonnull StructType left, @Nonnull StructType right) {
|
||||
StructField[] oneSchemaFields = left.fields();
|
||||
StructField[] anotherSchemaFields = right.fields();
|
||||
|
||||
if (oneSchemaFields.length != anotherSchemaFields.length) {
|
||||
return false;
|
||||
}
|
||||
|
||||
for (int i = 0; i < oneSchemaFields.length; ++i) {
|
||||
StructField oneField = oneSchemaFields[i];
|
||||
StructField anotherField = anotherSchemaFields[i];
|
||||
// NOTE: Metadata is deliberately omitted from comparison
|
||||
if (!Objects.equals(oneField.name(), anotherField.name())
|
||||
|| !areCompatible(oneField.dataType(), anotherField.dataType())
|
||||
|| oneField.nullable() != anotherField.nullable()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private static <T> HashSet<T> newHashSet(T... ts) {
|
||||
return new HashSet<>(Arrays.asList(ts));
|
||||
}
|
||||
}
|
||||
@@ -1,356 +0,0 @@
|
||||
/*
|
||||
* 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;
|
||||
String previousJobDescription = sc.getLocalProperty(SPARK_JOB_DESCRIPTION);
|
||||
try {
|
||||
jsc.setJobDescription("Listing parquet column statistics");
|
||||
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(new ArrayList<>(fileToStatsListMap.values()), 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) {
|
||||
rows.add(currentColRangeMetaData.getMinValueAsString());
|
||||
rows.add(currentColRangeMetaData.getMaxValueAsString());
|
||||
} else if (colType instanceof DecimalType) {
|
||||
rows.add(new BigDecimal(currentColRangeMetaData.getMinValueAsString()));
|
||||
rows.add(new BigDecimal(currentColRangeMetaData.getMaxValueAsString()));
|
||||
} else if (colType instanceof DateType) {
|
||||
rows.add(java.sql.Date.valueOf(currentColRangeMetaData.getMinValueAsString()));
|
||||
rows.add(java.sql.Date.valueOf(currentColRangeMetaData.getMaxValueAsString()));
|
||||
} 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 there's currently no index, create one
|
||||
if (!fs.exists(new Path(indexPath))) {
|
||||
statisticsDF.repartition(1).write().mode("overwrite").save(savePath.toString());
|
||||
return;
|
||||
}
|
||||
|
||||
// Otherwise, clean up all indexes but the most recent one
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -20,7 +20,6 @@ package org.apache.hudi
|
||||
|
||||
import java.nio.ByteBuffer
|
||||
import java.sql.{Date, Timestamp}
|
||||
import java.util
|
||||
|
||||
import org.apache.avro.Conversions.DecimalConversion
|
||||
import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis}
|
||||
@@ -318,7 +317,7 @@ object AvroConversionHelper {
|
||||
} else {
|
||||
val sourceArray = item.asInstanceOf[Seq[Any]]
|
||||
val sourceArraySize = sourceArray.size
|
||||
val targetList = new util.ArrayList[Any](sourceArraySize)
|
||||
val targetList = new java.util.ArrayList[Any](sourceArraySize)
|
||||
var idx = 0
|
||||
while (idx < sourceArraySize) {
|
||||
targetList.add(elementConverter(sourceArray(idx)))
|
||||
@@ -336,7 +335,7 @@ object AvroConversionHelper {
|
||||
if (item == null) {
|
||||
null
|
||||
} else {
|
||||
val javaMap = new util.HashMap[String, Any]()
|
||||
val javaMap = new java.util.HashMap[String, Any]()
|
||||
item.asInstanceOf[Map[String, Any]].foreach { case (key, value) =>
|
||||
javaMap.put(key, valueConverter(value))
|
||||
}
|
||||
|
||||
@@ -287,43 +287,4 @@ 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 <- cols.indices) {
|
||||
selectsql = selectsql + s" if (${leftTable}.${cols(i)} 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
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,8 +18,6 @@
|
||||
|
||||
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}
|
||||
@@ -197,22 +195,22 @@ class RawDecisionBound[K : Ordering : ClassTag](ordering: Ordering[K]) extends S
|
||||
// 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])
|
||||
(l, x) => java.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])
|
||||
(l, x) => java.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])
|
||||
(l, x) => java.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])
|
||||
(l, x) => java.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])
|
||||
(l, x) => java.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])
|
||||
(l, x) => java.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])
|
||||
(l, x) => java.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)
|
||||
(l, x) => java.util.Arrays.binarySearch(l.asInstanceOf[Array[AnyRef]], x, comparator)
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user