[HUDI-3172] Refactor hudi existing modules to make more code reuse in V2 Implementation (#4514)
* Introduce hudi-spark3-common and hudi-spark2-common modules to place classes that would be reused in different spark versions, also introduce hudi-spark3.1.x to support spark 3.1.x. * Introduce hudi format under hudi-spark2, hudi-spark3, hudi-spark3.1.x modules and change the hudi format in original hudi-spark module to hudi_v1 format. * Manually tested on Spark 3.1.2 and Spark 3.2.0 SQL. * Added a README.md file under hudi-spark-datasource module.
This commit is contained in:
@@ -0,0 +1,166 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.keygen.BuiltinKeyGenerator;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.sql.Column;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.api.java.UDF1;
|
||||
import org.apache.spark.sql.functions;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
import static org.apache.spark.sql.functions.callUDF;
|
||||
|
||||
/**
|
||||
* Helper class to assist in preparing {@link Dataset<Row>}s for bulk insert with datasource implementation.
|
||||
*/
|
||||
public class HoodieDatasetBulkInsertHelper {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieDatasetBulkInsertHelper.class);
|
||||
|
||||
private static final String RECORD_KEY_UDF_FN = "hudi_recordkey_gen_function_";
|
||||
private static final String PARTITION_PATH_UDF_FN = "hudi_partition_gen_function_";
|
||||
|
||||
/**
|
||||
* Prepares input hoodie spark dataset for bulk insert. It does the following steps.
|
||||
* 1. Uses KeyGenerator to generate hoodie record keys and partition path.
|
||||
* 2. Add hoodie columns to input spark dataset.
|
||||
* 3. Reorders input dataset columns so that hoodie columns appear in the beginning.
|
||||
* 4. Sorts input dataset by hoodie partition path and record key
|
||||
*
|
||||
* @param sqlContext SQL Context
|
||||
* @param config Hoodie Write Config
|
||||
* @param rows Spark Input dataset
|
||||
* @return hoodie dataset which is ready for bulk insert.
|
||||
*/
|
||||
public static Dataset<Row> prepareHoodieDatasetForBulkInsert(SQLContext sqlContext,
|
||||
HoodieWriteConfig config, Dataset<Row> rows, String structName, String recordNamespace,
|
||||
BulkInsertPartitioner<Dataset<Row>> bulkInsertPartitionerRows,
|
||||
boolean isGlobalIndex, boolean dropPartitionColumns) {
|
||||
List<Column> originalFields =
|
||||
Arrays.stream(rows.schema().fields()).map(f -> new Column(f.name())).collect(Collectors.toList());
|
||||
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(config.getProps());
|
||||
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key());
|
||||
BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
|
||||
String tableName = properties.getString(HoodieWriteConfig.TBL_NAME.key());
|
||||
String recordKeyUdfFn = RECORD_KEY_UDF_FN + tableName;
|
||||
String partitionPathUdfFn = PARTITION_PATH_UDF_FN + tableName;
|
||||
sqlContext.udf().register(recordKeyUdfFn, (UDF1<Row, String>) keyGenerator::getRecordKey, DataTypes.StringType);
|
||||
sqlContext.udf().register(partitionPathUdfFn, (UDF1<Row, String>) keyGenerator::getPartitionPath, DataTypes.StringType);
|
||||
|
||||
final Dataset<Row> rowDatasetWithRecordKeys = rows.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD,
|
||||
callUDF(recordKeyUdfFn, org.apache.spark.sql.functions.struct(
|
||||
JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq())));
|
||||
|
||||
final Dataset<Row> rowDatasetWithRecordKeysAndPartitionPath =
|
||||
rowDatasetWithRecordKeys.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
|
||||
callUDF(partitionPathUdfFn,
|
||||
org.apache.spark.sql.functions.struct(
|
||||
JavaConverters.collectionAsScalaIterableConverter(originalFields).asScala().toSeq())));
|
||||
|
||||
// Add other empty hoodie fields which will be populated before writing to parquet.
|
||||
Dataset<Row> rowDatasetWithHoodieColumns =
|
||||
rowDatasetWithRecordKeysAndPartitionPath.withColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType));
|
||||
|
||||
Dataset<Row> processedDf = rowDatasetWithHoodieColumns;
|
||||
if (dropPartitionColumns) {
|
||||
String partitionColumns = String.join(",", keyGenerator.getPartitionPathFields());
|
||||
for (String partitionField: keyGenerator.getPartitionPathFields()) {
|
||||
originalFields.remove(new Column(partitionField));
|
||||
}
|
||||
processedDf = rowDatasetWithHoodieColumns.drop(partitionColumns);
|
||||
}
|
||||
Dataset<Row> dedupedDf = processedDf;
|
||||
if (config.shouldCombineBeforeInsert()) {
|
||||
dedupedDf = SparkRowWriteHelper.newInstance().deduplicateRows(processedDf, config.getPreCombineField(), isGlobalIndex);
|
||||
}
|
||||
|
||||
List<Column> orderedFields = Stream.concat(HoodieRecord.HOODIE_META_COLUMNS.stream().map(Column::new),
|
||||
originalFields.stream()).collect(Collectors.toList());
|
||||
Dataset<Row> colOrderedDataset = dedupedDf.select(
|
||||
JavaConverters.collectionAsScalaIterableConverter(orderedFields).asScala().toSeq());
|
||||
|
||||
return bulkInsertPartitionerRows.repartitionRecords(colOrderedDataset, config.getBulkInsertShuffleParallelism());
|
||||
}
|
||||
|
||||
/**
|
||||
* Add empty meta fields and reorder such that meta fields are at the beginning.
|
||||
*
|
||||
* @param rows
|
||||
* @return
|
||||
*/
|
||||
public static Dataset<Row> prepareHoodieDatasetForBulkInsertWithoutMetaFields(Dataset<Row> rows) {
|
||||
// add empty meta cols.
|
||||
Dataset<Row> rowsWithMetaCols = rows
|
||||
.withColumn(HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.RECORD_KEY_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.PARTITION_PATH_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType))
|
||||
.withColumn(HoodieRecord.FILENAME_METADATA_FIELD,
|
||||
functions.lit("").cast(DataTypes.StringType));
|
||||
|
||||
List<Column> originalFields =
|
||||
Arrays.stream(rowsWithMetaCols.schema().fields())
|
||||
.filter(field -> !HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(field.name()))
|
||||
.map(f -> new Column(f.name())).collect(Collectors.toList());
|
||||
|
||||
List<Column> metaFields =
|
||||
Arrays.stream(rowsWithMetaCols.schema().fields())
|
||||
.filter(field -> HoodieRecord.HOODIE_META_COLUMNS_WITH_OPERATION.contains(field.name()))
|
||||
.map(f -> new Column(f.name())).collect(Collectors.toList());
|
||||
|
||||
// reorder such that all meta columns are at the beginning followed by original columns
|
||||
List<Column> allCols = new ArrayList<>();
|
||||
allCols.addAll(metaFields);
|
||||
allCols.addAll(originalFields);
|
||||
|
||||
return rowsWithMetaCols.select(
|
||||
JavaConverters.collectionAsScalaIterableConverter(allCols).asScala().toSeq());
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,81 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.apache.spark.api.java.function.MapFunction;
|
||||
import org.apache.spark.api.java.function.ReduceFunction;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Encoders;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
|
||||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
||||
import org.apache.spark.sql.catalyst.expressions.Attribute;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
import scala.collection.JavaConversions;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
/**
|
||||
* Helper class to assist in deduplicating Rows for BulkInsert with Rows.
|
||||
*/
|
||||
public class SparkRowWriteHelper {
|
||||
|
||||
private SparkRowWriteHelper() {
|
||||
}
|
||||
|
||||
private static class WriteHelperHolder {
|
||||
private static final SparkRowWriteHelper SPARK_WRITE_HELPER = new SparkRowWriteHelper();
|
||||
}
|
||||
|
||||
public static SparkRowWriteHelper newInstance() {
|
||||
return SparkRowWriteHelper.WriteHelperHolder.SPARK_WRITE_HELPER;
|
||||
}
|
||||
|
||||
public Dataset<Row> deduplicateRows(Dataset<Row> inputDf, String preCombineField, boolean isGlobalIndex) {
|
||||
ExpressionEncoder encoder = getEncoder(inputDf.schema());
|
||||
|
||||
return inputDf.groupByKey(
|
||||
(MapFunction<Row, String>) value ->
|
||||
isGlobalIndex ? (value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)) :
|
||||
(value.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD) + "+" + value.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD)), Encoders.STRING())
|
||||
.reduceGroups((ReduceFunction<Row>) (v1, v2) -> {
|
||||
if (((Comparable) v1.getAs(preCombineField)).compareTo(((Comparable) v2.getAs(preCombineField))) >= 0) {
|
||||
return v1;
|
||||
} else {
|
||||
return v2;
|
||||
}
|
||||
}
|
||||
).map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder);
|
||||
}
|
||||
|
||||
private ExpressionEncoder getEncoder(StructType schema) {
|
||||
List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
|
||||
.map(Attribute::toAttribute).collect(Collectors.toList());
|
||||
return RowEncoder.apply(schema)
|
||||
.resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(),
|
||||
SimpleAnalyzer$.MODULE$);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.async;
|
||||
|
||||
import org.apache.hudi.client.AbstractClusteringClient;
|
||||
import org.apache.hudi.client.AbstractHoodieWriteClient;
|
||||
import org.apache.hudi.client.HoodieSparkClusteringClient;
|
||||
|
||||
/**
|
||||
* Async clustering service for Spark structured streaming.
|
||||
* Here, async clustering is run in daemon mode to prevent blocking shutting down the Spark application.
|
||||
*/
|
||||
public class SparkStreamingAsyncClusteringService extends AsyncClusteringService {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public SparkStreamingAsyncClusteringService(AbstractHoodieWriteClient writeClient) {
|
||||
super(writeClient, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractClusteringClient createClusteringClient(AbstractHoodieWriteClient client) {
|
||||
return new HoodieSparkClusteringClient(client);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* 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.async;
|
||||
|
||||
import org.apache.hudi.client.AbstractCompactor;
|
||||
import org.apache.hudi.client.AbstractHoodieWriteClient;
|
||||
import org.apache.hudi.client.HoodieSparkCompactor;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
|
||||
/**
|
||||
* Async Compaction Service used by Structured Streaming. Here, async compaction is run in daemon mode to prevent
|
||||
* blocking shutting down the Spark application.
|
||||
*/
|
||||
public class SparkStreamingAsyncCompactService extends AsyncCompactService {
|
||||
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public SparkStreamingAsyncCompactService(HoodieEngineContext context, AbstractHoodieWriteClient client) {
|
||||
super(context, client, true);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected AbstractCompactor createCompactor(AbstractHoodieWriteClient client) {
|
||||
return new HoodieSparkCompactor(client, this.context);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,37 @@
|
||||
/*
|
||||
* 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.sql;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
/***
|
||||
* A interface for CodeGen to execute expressions on the record.
|
||||
*/
|
||||
public interface IExpressionEvaluator {
|
||||
|
||||
/**
|
||||
* Evaluate the result of the expressions based on the record.
|
||||
*/
|
||||
GenericRecord eval(IndexedRecord record);
|
||||
|
||||
/**
|
||||
* Get the code of the expressions. This is used for debug.
|
||||
*/
|
||||
String getCode();
|
||||
}
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.sql;
|
||||
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* Insert mode for insert into pk-table.
|
||||
*/
|
||||
public enum InsertMode {
|
||||
/**
|
||||
* In upsert mode for insert into, duplicate record on primary key
|
||||
* will be updated.This is the default insert mode for pk-table.
|
||||
*/
|
||||
UPSERT("upsert"),
|
||||
/**
|
||||
* In strict mode for insert into, we do the pk uniqueness guarantee
|
||||
* for COW pk-table.
|
||||
* For MOR pk-table, it has the same behavior with "upsert" mode.
|
||||
*/
|
||||
STRICT("strict"),
|
||||
/**
|
||||
* In non-strict mode for insert into, we use insert operation
|
||||
* to write data which allow writing the duplicate record.
|
||||
*/
|
||||
NON_STRICT("non-strict")
|
||||
;
|
||||
|
||||
private String value;
|
||||
|
||||
InsertMode(String value) {
|
||||
this.value = value;
|
||||
}
|
||||
|
||||
public String value() {
|
||||
return value;
|
||||
}
|
||||
|
||||
public static InsertMode of(String value) {
|
||||
switch (value.toLowerCase(Locale.ROOT)) {
|
||||
case "upsert":
|
||||
return UPSERT;
|
||||
case "strict":
|
||||
return STRICT;
|
||||
case "non-strict":
|
||||
return NON_STRICT;
|
||||
default:
|
||||
throw new AssertionError("UnSupport Insert Mode: " + value);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,19 @@
|
||||
|
||||
# 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.
|
||||
|
||||
|
||||
org.apache.hudi.DefaultSource
|
||||
@@ -0,0 +1,272 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import org.apache.hudi.DataSourceReadOptions._
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
|
||||
import org.apache.hudi.DataSourceWriteOptions.{BOOTSTRAP_OPERATION_OPT_VAL, OPERATION}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hadoop.HoodieROTablePathFilter
|
||||
|
||||
import org.apache.log4j.LogManager
|
||||
|
||||
import org.apache.spark.sql.execution.datasources.{DataSource, FileStatusCache, HadoopFsRelation}
|
||||
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.execution.streaming.{Sink, Source}
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieStreamSource
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.streaming.OutputMode
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* Hoodie Spark Datasource, for reading and writing hoodie tables
|
||||
*
|
||||
*/
|
||||
class DefaultSource extends RelationProvider
|
||||
with SchemaRelationProvider
|
||||
with CreatableRelationProvider
|
||||
with DataSourceRegister
|
||||
with StreamSinkProvider
|
||||
with StreamSourceProvider
|
||||
with Serializable {
|
||||
|
||||
SparkSession.getActiveSession.foreach { spark =>
|
||||
val sparkVersion = spark.version
|
||||
if (sparkVersion.startsWith("0.") || sparkVersion.startsWith("1.") || sparkVersion.startsWith("2.")) {
|
||||
// Enable "passPartitionByAsOptions" to support "write.partitionBy(...)"
|
||||
spark.conf.set("spark.sql.legacy.sources.write.passPartitionByAsOptions", "true")
|
||||
}
|
||||
}
|
||||
|
||||
private val log = LogManager.getLogger(classOf[DefaultSource])
|
||||
|
||||
override def createRelation(sqlContext: SQLContext,
|
||||
parameters: Map[String, String]): BaseRelation = {
|
||||
createRelation(sqlContext, parameters, null)
|
||||
}
|
||||
|
||||
override def createRelation(sqlContext: SQLContext,
|
||||
optParams: Map[String, String],
|
||||
schema: StructType): BaseRelation = {
|
||||
// Add default options for unspecified read options keys.
|
||||
val parameters = DataSourceOptionsHelper.parametersWithReadDefaults(optParams)
|
||||
|
||||
val path = parameters.get("path")
|
||||
val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS.key)
|
||||
if (path.isEmpty && readPathsStr.isEmpty) {
|
||||
throw new HoodieException(s"'path' or '$READ_PATHS' or both must be specified.")
|
||||
}
|
||||
|
||||
val readPaths = readPathsStr.map(p => p.split(",").toSeq).getOrElse(Seq())
|
||||
val allPaths = path.map(p => Seq(p)).getOrElse(Seq()) ++ readPaths
|
||||
|
||||
val fs = FSUtils.getFs(allPaths.head, sqlContext.sparkContext.hadoopConfiguration)
|
||||
// Use the HoodieFileIndex only if the 'path' is not globbed.
|
||||
// Or else we use the original way to read hoodie table.
|
||||
val enableFileIndex = optParams.get(ENABLE_HOODIE_FILE_INDEX.key)
|
||||
.map(_.toBoolean).getOrElse(ENABLE_HOODIE_FILE_INDEX.defaultValue)
|
||||
val useHoodieFileIndex = enableFileIndex && path.isDefined && !path.get.contains("*") &&
|
||||
!parameters.contains(DataSourceReadOptions.READ_PATHS.key)
|
||||
val globPaths = if (useHoodieFileIndex) {
|
||||
None
|
||||
} else {
|
||||
Some(HoodieSparkUtils.checkAndGlobPathIfNecessary(allPaths, fs))
|
||||
}
|
||||
// Get the table base path
|
||||
val tablePath = if (globPaths.isDefined) {
|
||||
DataSourceUtils.getTablePath(fs, globPaths.get.toArray)
|
||||
} else {
|
||||
DataSourceUtils.getTablePath(fs, Array(new Path(path.get)))
|
||||
}
|
||||
log.info("Obtained hudi table path: " + tablePath)
|
||||
|
||||
val metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(tablePath).build()
|
||||
val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent
|
||||
val tableType = metaClient.getTableType
|
||||
val queryType = parameters(QUERY_TYPE.key)
|
||||
|
||||
log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType, queryType is: $queryType")
|
||||
|
||||
(tableType, queryType, isBootstrappedTable) match {
|
||||
case (COPY_ON_WRITE, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) |
|
||||
(COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) |
|
||||
(MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) =>
|
||||
getBaseFileOnlyView(useHoodieFileIndex, sqlContext, parameters, schema, tablePath,
|
||||
readPaths, metaClient)
|
||||
|
||||
case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) =>
|
||||
new IncrementalRelation(sqlContext, parameters, schema, metaClient)
|
||||
|
||||
case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL, false) =>
|
||||
new MergeOnReadSnapshotRelation(sqlContext, parameters, schema, globPaths, metaClient)
|
||||
|
||||
case (MERGE_ON_READ, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) =>
|
||||
new MergeOnReadIncrementalRelation(sqlContext, parameters, schema, metaClient)
|
||||
|
||||
case (_, _, true) =>
|
||||
new HoodieBootstrapRelation(sqlContext, schema, globPaths, metaClient, parameters)
|
||||
|
||||
case (_, _, _) =>
|
||||
throw new HoodieException(s"Invalid query type : $queryType for tableType: $tableType," +
|
||||
s"isBootstrappedTable: $isBootstrappedTable ")
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This DataSource API is used for writing the DataFrame at the destination. For now, we are returning a dummy
|
||||
* relation here because Spark does not really make use of the relation returned, and just returns an empty
|
||||
* dataset at [[org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand.run()]]. This saves us the cost
|
||||
* of creating and returning a parquet relation here.
|
||||
*
|
||||
* TODO: Revisit to return a concrete relation here when we support CREATE TABLE AS for Hudi with DataSource API.
|
||||
* That is the only case where Spark seems to actually need a relation to be returned here
|
||||
* [[DataSource.writeAndRead()]]
|
||||
*
|
||||
* @param sqlContext Spark SQL Context
|
||||
* @param mode Mode for saving the DataFrame at the destination
|
||||
* @param optParams Parameters passed as part of the DataFrame write operation
|
||||
* @param df Spark DataFrame to be written
|
||||
* @return Spark Relation
|
||||
*/
|
||||
override def createRelation(sqlContext: SQLContext,
|
||||
mode: SaveMode,
|
||||
optParams: Map[String, String],
|
||||
df: DataFrame): BaseRelation = {
|
||||
val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*)
|
||||
|
||||
if (optParams.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)) {
|
||||
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, optParams, dfWithoutMetaCols)
|
||||
} else {
|
||||
HoodieSparkSqlWriter.write(sqlContext, mode, optParams, dfWithoutMetaCols)
|
||||
}
|
||||
new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema)
|
||||
}
|
||||
|
||||
override def createSink(sqlContext: SQLContext,
|
||||
optParams: Map[String, String],
|
||||
partitionColumns: Seq[String],
|
||||
outputMode: OutputMode): Sink = {
|
||||
new HoodieStreamingSink(
|
||||
sqlContext,
|
||||
optParams,
|
||||
partitionColumns,
|
||||
outputMode)
|
||||
}
|
||||
|
||||
override def shortName(): String = "hudi_v1"
|
||||
|
||||
private def getBaseFileOnlyView(useHoodieFileIndex: Boolean,
|
||||
sqlContext: SQLContext,
|
||||
optParams: Map[String, String],
|
||||
schema: StructType,
|
||||
tablePath: String,
|
||||
extraReadPaths: Seq[String],
|
||||
metaClient: HoodieTableMetaClient): BaseRelation = {
|
||||
log.info("Loading Base File Only View with options :" + optParams)
|
||||
val (tableFileFormat, formatClassName) = metaClient.getTableConfig.getBaseFileFormat match {
|
||||
case HoodieFileFormat.PARQUET => (new ParquetFileFormat, "parquet")
|
||||
case HoodieFileFormat.ORC => (new OrcFileFormat, "orc")
|
||||
}
|
||||
|
||||
if (useHoodieFileIndex) {
|
||||
|
||||
val fileIndex = HoodieFileIndex(sqlContext.sparkSession, metaClient,
|
||||
if (schema == null) Option.empty[StructType] else Some(schema),
|
||||
optParams, FileStatusCache.getOrCreate(sqlContext.sparkSession))
|
||||
|
||||
HadoopFsRelation(
|
||||
fileIndex,
|
||||
fileIndex.partitionSchema,
|
||||
fileIndex.dataSchema,
|
||||
bucketSpec = None,
|
||||
fileFormat = tableFileFormat,
|
||||
optParams)(sqlContext.sparkSession)
|
||||
} else {
|
||||
// this is just effectively RO view only, where `path` can contain a mix of
|
||||
// non-hoodie/hoodie path files. set the path filter up
|
||||
sqlContext.sparkContext.hadoopConfiguration.setClass(
|
||||
"mapreduce.input.pathFilter.class",
|
||||
classOf[HoodieROTablePathFilter],
|
||||
classOf[org.apache.hadoop.fs.PathFilter])
|
||||
|
||||
val specifySchema = if (schema == null) {
|
||||
// Load the schema from the commit meta data.
|
||||
// Here we should specify the schema to the latest commit schema since
|
||||
// the table schema evolution.
|
||||
val tableSchemaResolver = new TableSchemaResolver(metaClient)
|
||||
try {
|
||||
Some(AvroConversionUtils.convertAvroSchemaToStructType(tableSchemaResolver.getTableAvroSchema))
|
||||
} catch {
|
||||
case _: Throwable =>
|
||||
None // If there is no commit in the table, we can not get the schema
|
||||
// with tableSchemaResolver, return None here.
|
||||
}
|
||||
} else {
|
||||
Some(schema)
|
||||
}
|
||||
// simply return as a regular relation
|
||||
DataSource.apply(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
paths = extraReadPaths,
|
||||
userSpecifiedSchema = specifySchema,
|
||||
className = formatClassName,
|
||||
options = optParams)
|
||||
.resolveRelation()
|
||||
}
|
||||
}
|
||||
|
||||
override def sourceSchema(sqlContext: SQLContext,
|
||||
schema: Option[StructType],
|
||||
providerName: String,
|
||||
parameters: Map[String, String]): (String, StructType) = {
|
||||
val path = parameters.get("path")
|
||||
if (path.isEmpty || path.get == null) {
|
||||
throw new HoodieException(s"'path' must be specified.")
|
||||
}
|
||||
val metaClient = HoodieTableMetaClient.builder().setConf(
|
||||
sqlContext.sparkSession.sessionState.newHadoopConf()).setBasePath(path.get).build()
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val sqlSchema =
|
||||
try {
|
||||
val avroSchema = schemaResolver.getTableAvroSchema
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(avroSchema)
|
||||
} catch {
|
||||
case _: Exception =>
|
||||
require(schema.isDefined, "Fail to resolve source schema")
|
||||
schema.get
|
||||
}
|
||||
(shortName(), sqlSchema)
|
||||
}
|
||||
|
||||
override def createSource(sqlContext: SQLContext,
|
||||
metadataPath: String,
|
||||
schema: Option[StructType],
|
||||
providerName: String,
|
||||
parameters: Map[String, String]): Source = {
|
||||
new HoodieStreamSource(sqlContext, metadataPath, schema, parameters)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.spark.{Partition, TaskContext}
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch
|
||||
|
||||
class HoodieBootstrapRDD(@transient spark: SparkSession,
|
||||
dataReadFunction: PartitionedFile => Iterator[Any],
|
||||
skeletonReadFunction: PartitionedFile => Iterator[Any],
|
||||
regularReadFunction: PartitionedFile => Iterator[Any],
|
||||
dataSchema: StructType,
|
||||
skeletonSchema: StructType,
|
||||
requiredColumns: Array[String],
|
||||
tableState: HoodieBootstrapTableState)
|
||||
extends RDD[InternalRow](spark.sparkContext, Nil) {
|
||||
|
||||
override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
|
||||
val bootstrapPartition = split.asInstanceOf[HoodieBootstrapPartition]
|
||||
|
||||
if (log.isDebugEnabled) {
|
||||
if (bootstrapPartition.split.skeletonFile.isDefined) {
|
||||
logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: "
|
||||
+ bootstrapPartition.split.dataFile.filePath + ", Skeleton File: "
|
||||
+ bootstrapPartition.split.skeletonFile.get.filePath)
|
||||
} else {
|
||||
logDebug("Got Split => Index: " + bootstrapPartition.index + ", Data File: "
|
||||
+ bootstrapPartition.split.dataFile.filePath)
|
||||
}
|
||||
}
|
||||
|
||||
var partitionedFileIterator: Iterator[InternalRow] = null
|
||||
|
||||
if (bootstrapPartition.split.skeletonFile.isDefined) {
|
||||
// It is a bootstrap split. Check both skeleton and data files.
|
||||
if (dataSchema.isEmpty) {
|
||||
// No data column to fetch, hence fetch only from skeleton file
|
||||
partitionedFileIterator = read(bootstrapPartition.split.skeletonFile.get, skeletonReadFunction)
|
||||
} else if (skeletonSchema.isEmpty) {
|
||||
// No metadata column to fetch, hence fetch only from data file
|
||||
partitionedFileIterator = read(bootstrapPartition.split.dataFile, dataReadFunction)
|
||||
} else {
|
||||
// Fetch from both data and skeleton file, and merge
|
||||
val dataFileIterator = read(bootstrapPartition.split.dataFile, dataReadFunction)
|
||||
val skeletonFileIterator = read(bootstrapPartition.split.skeletonFile.get, skeletonReadFunction)
|
||||
partitionedFileIterator = merge(skeletonFileIterator, dataFileIterator)
|
||||
}
|
||||
} else {
|
||||
partitionedFileIterator = read(bootstrapPartition.split.dataFile, regularReadFunction)
|
||||
}
|
||||
partitionedFileIterator
|
||||
}
|
||||
|
||||
def merge(skeletonFileIterator: Iterator[InternalRow], dataFileIterator: Iterator[InternalRow])
|
||||
: Iterator[InternalRow] = {
|
||||
new Iterator[InternalRow] {
|
||||
override def hasNext: Boolean = dataFileIterator.hasNext && skeletonFileIterator.hasNext
|
||||
override def next(): InternalRow = {
|
||||
mergeInternalRow(skeletonFileIterator.next(), dataFileIterator.next())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
def mergeInternalRow(skeletonRow: InternalRow, dataRow: InternalRow): InternalRow = {
|
||||
val skeletonArr = skeletonRow.copy().toSeq(skeletonSchema)
|
||||
val dataArr = dataRow.copy().toSeq(dataSchema)
|
||||
// We need to return it in the order requested
|
||||
val mergedArr = requiredColumns.map(col => {
|
||||
if (skeletonSchema.fieldNames.contains(col)) {
|
||||
val idx = skeletonSchema.fieldIndex(col)
|
||||
skeletonArr(idx)
|
||||
} else {
|
||||
val idx = dataSchema.fieldIndex(col)
|
||||
dataArr(idx)
|
||||
}
|
||||
})
|
||||
|
||||
logDebug("Merged data and skeleton values => " + mergedArr.mkString(","))
|
||||
val mergedRow = InternalRow.fromSeq(mergedArr)
|
||||
mergedRow
|
||||
}
|
||||
|
||||
def read(partitionedFile: PartitionedFile, readFileFunction: PartitionedFile => Iterator[Any])
|
||||
: Iterator[InternalRow] = {
|
||||
val fileIterator = readFileFunction(partitionedFile)
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
val rows = fileIterator.flatMap(_ match {
|
||||
case r: InternalRow => Seq(r)
|
||||
case b: ColumnarBatch => b.rowIterator().asScala
|
||||
})
|
||||
rows
|
||||
}
|
||||
|
||||
override protected def getPartitions: Array[Partition] = {
|
||||
tableState.files.zipWithIndex.map(file => {
|
||||
if (file._1.skeletonFile.isDefined) {
|
||||
logDebug("Forming partition with => Index: " + file._2 + ", Files: " + file._1.dataFile.filePath
|
||||
+ "," + file._1.skeletonFile.get.filePath)
|
||||
HoodieBootstrapPartition(file._2, file._1)
|
||||
} else {
|
||||
logDebug("Forming partition with => Index: " + file._2 + ", File: " + file._1.dataFile.filePath)
|
||||
HoodieBootstrapPartition(file._2, file._1)
|
||||
}
|
||||
}).toArray
|
||||
}
|
||||
}
|
||||
|
||||
case class HoodieBootstrapPartition(index: Int, split: HoodieBootstrapSplit) extends Partition
|
||||
@@ -0,0 +1,195 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.common.model.HoodieBaseFile
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.{Row, SQLContext}
|
||||
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* This is Spark relation that can be used for querying metadata/fully bootstrapped query hoodie tables, as well as
|
||||
* non-bootstrapped tables. It implements PrunedFilteredScan interface in order to support column pruning and filter
|
||||
* push-down. For metadata bootstrapped files, if we query columns from both metadata and actual data then it will
|
||||
* perform a merge of both to return the result.
|
||||
*
|
||||
* Caveat: Filter push-down does not work when querying both metadata and actual data columns over metadata
|
||||
* bootstrapped files, because then the metadata file and data file can return different number of rows causing errors
|
||||
* merging.
|
||||
*
|
||||
* @param _sqlContext Spark SQL Context
|
||||
* @param userSchema User specified schema in the datasource query
|
||||
* @param globPaths The global paths to query. If it not none, read from the globPaths,
|
||||
* else read data from tablePath using HoodiFileIndex.
|
||||
* @param metaClient Hoodie table meta client
|
||||
* @param optParams DataSource options passed by the user
|
||||
*/
|
||||
class HoodieBootstrapRelation(@transient val _sqlContext: SQLContext,
|
||||
val userSchema: StructType,
|
||||
val globPaths: Option[Seq[Path]],
|
||||
val metaClient: HoodieTableMetaClient,
|
||||
val optParams: Map[String, String]) extends BaseRelation
|
||||
with PrunedFilteredScan with Logging {
|
||||
|
||||
val skeletonSchema: StructType = HoodieSparkUtils.getMetaSchema
|
||||
var dataSchema: StructType = _
|
||||
var fullSchema: StructType = _
|
||||
|
||||
val fileIndex: HoodieBootstrapFileIndex = buildFileIndex()
|
||||
|
||||
override def sqlContext: SQLContext = _sqlContext
|
||||
|
||||
override val needConversion: Boolean = false
|
||||
|
||||
override def schema: StructType = inferFullSchema()
|
||||
|
||||
override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
|
||||
logInfo("Starting scan..")
|
||||
|
||||
// Compute splits
|
||||
val bootstrapSplits = fileIndex.files.map(hoodieBaseFile => {
|
||||
var skeletonFile: Option[PartitionedFile] = Option.empty
|
||||
var dataFile: PartitionedFile = null
|
||||
|
||||
if (hoodieBaseFile.getBootstrapBaseFile.isPresent) {
|
||||
skeletonFile = Option(PartitionedFile(InternalRow.empty, hoodieBaseFile.getPath, 0, hoodieBaseFile.getFileLen))
|
||||
dataFile = PartitionedFile(InternalRow.empty, hoodieBaseFile.getBootstrapBaseFile.get().getPath, 0,
|
||||
hoodieBaseFile.getBootstrapBaseFile.get().getFileLen)
|
||||
} else {
|
||||
dataFile = PartitionedFile(InternalRow.empty, hoodieBaseFile.getPath, 0, hoodieBaseFile.getFileLen)
|
||||
}
|
||||
HoodieBootstrapSplit(dataFile, skeletonFile)
|
||||
})
|
||||
val tableState = HoodieBootstrapTableState(bootstrapSplits)
|
||||
|
||||
// Get required schemas for column pruning
|
||||
var requiredDataSchema = StructType(Seq())
|
||||
var requiredSkeletonSchema = StructType(Seq())
|
||||
// requiredColsSchema is the schema of requiredColumns, note that requiredColumns is in a random order
|
||||
// so requiredColsSchema is not always equal to (requiredSkeletonSchema.fields ++ requiredDataSchema.fields)
|
||||
var requiredColsSchema = StructType(Seq())
|
||||
requiredColumns.foreach(col => {
|
||||
var field = dataSchema.find(_.name == col)
|
||||
if (field.isDefined) {
|
||||
requiredDataSchema = requiredDataSchema.add(field.get)
|
||||
} else {
|
||||
field = skeletonSchema.find(_.name == col)
|
||||
requiredSkeletonSchema = requiredSkeletonSchema.add(field.get)
|
||||
}
|
||||
requiredColsSchema = requiredColsSchema.add(field.get)
|
||||
})
|
||||
|
||||
// Prepare readers for reading data file and skeleton files
|
||||
val dataReadFunction = new ParquetFileFormat()
|
||||
.buildReaderWithPartitionValues(
|
||||
sparkSession = _sqlContext.sparkSession,
|
||||
dataSchema = dataSchema,
|
||||
partitionSchema = StructType(Seq.empty),
|
||||
requiredSchema = requiredDataSchema,
|
||||
filters = if (requiredSkeletonSchema.isEmpty) filters else Seq() ,
|
||||
options = Map.empty,
|
||||
hadoopConf = _sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
)
|
||||
|
||||
val skeletonReadFunction = new ParquetFileFormat()
|
||||
.buildReaderWithPartitionValues(
|
||||
sparkSession = _sqlContext.sparkSession,
|
||||
dataSchema = skeletonSchema,
|
||||
partitionSchema = StructType(Seq.empty),
|
||||
requiredSchema = requiredSkeletonSchema,
|
||||
filters = if (requiredDataSchema.isEmpty) filters else Seq(),
|
||||
options = Map.empty,
|
||||
hadoopConf = _sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
)
|
||||
|
||||
val regularReadFunction = new ParquetFileFormat()
|
||||
.buildReaderWithPartitionValues(
|
||||
sparkSession = _sqlContext.sparkSession,
|
||||
dataSchema = fullSchema,
|
||||
partitionSchema = StructType(Seq.empty),
|
||||
requiredSchema = requiredColsSchema,
|
||||
filters = filters,
|
||||
options = Map.empty,
|
||||
hadoopConf = _sqlContext.sparkSession.sessionState.newHadoopConf())
|
||||
|
||||
val rdd = new HoodieBootstrapRDD(_sqlContext.sparkSession, dataReadFunction, skeletonReadFunction,
|
||||
regularReadFunction, requiredDataSchema, requiredSkeletonSchema, requiredColumns, tableState)
|
||||
rdd.asInstanceOf[RDD[Row]]
|
||||
}
|
||||
|
||||
def inferFullSchema(): StructType = {
|
||||
if (fullSchema == null) {
|
||||
logInfo("Inferring schema..")
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val tableSchema = schemaResolver.getTableAvroSchemaWithoutMetadataFields
|
||||
dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema)
|
||||
fullSchema = StructType(skeletonSchema.fields ++ dataSchema.fields)
|
||||
}
|
||||
fullSchema
|
||||
}
|
||||
|
||||
def buildFileIndex(): HoodieBootstrapFileIndex = {
|
||||
logInfo("Building file index..")
|
||||
val fileStatuses = if (globPaths.isDefined) {
|
||||
// Load files from the global paths if it has defined to be compatible with the original mode
|
||||
val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(_sqlContext.sparkSession, globPaths.get)
|
||||
inMemoryFileIndex.allFiles()
|
||||
} else { // Load files by the HoodieFileIndex.
|
||||
HoodieFileIndex(sqlContext.sparkSession, metaClient, Some(schema), optParams,
|
||||
FileStatusCache.getOrCreate(sqlContext.sparkSession)).allFiles
|
||||
}
|
||||
if (fileStatuses.isEmpty) {
|
||||
throw new HoodieException("No files found for reading in user provided path.")
|
||||
}
|
||||
|
||||
val fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline.getCommitsTimeline
|
||||
.filterCompletedInstants, fileStatuses.toArray)
|
||||
val latestFiles: List[HoodieBaseFile] = fsView.getLatestBaseFiles.iterator().asScala.toList
|
||||
|
||||
if (log.isDebugEnabled) {
|
||||
latestFiles.foreach(file => {
|
||||
logDebug("Printing indexed files:")
|
||||
if (file.getBootstrapBaseFile.isPresent) {
|
||||
logDebug("Skeleton File: " + file.getPath + ", Data File: " + file.getBootstrapBaseFile.get().getPath)
|
||||
} else {
|
||||
logDebug("Regular Hoodie File: " + file.getPath)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
HoodieBootstrapFileIndex(latestFiles)
|
||||
}
|
||||
}
|
||||
|
||||
case class HoodieBootstrapFileIndex(files: List[HoodieBaseFile])
|
||||
|
||||
case class HoodieBootstrapTableState(files: List[HoodieBootstrapSplit])
|
||||
|
||||
case class HoodieBootstrapSplit(dataFile: PartitionedFile, skeletonFile: Option[PartitionedFile])
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.spark.sql.SQLContext
|
||||
import org.apache.spark.sql.sources.BaseRelation
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
/**
|
||||
* This is a dummy Spark relation that can be used if needed to return a place holder relation that does not get used.
|
||||
*
|
||||
* @param sqlContext Spark SQL Context
|
||||
* @param userSchema Users data schema
|
||||
*/
|
||||
class HoodieEmptyRelation(val sqlContext: SQLContext,
|
||||
val userSchema: StructType) extends BaseRelation {
|
||||
|
||||
override def schema: StructType = userSchema
|
||||
}
|
||||
@@ -0,0 +1,653 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.FileSlice
|
||||
import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
|
||||
import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, BoundReference, Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory}
|
||||
import org.apache.spark.sql.hudi.DataSkippingUtils.createColumnStatsIndexFilterExpr
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{AnalysisException, Column, SparkSession}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
||||
/**
|
||||
* A file index which support partition prune for hoodie snapshot and read-optimized query.
|
||||
*
|
||||
* Main steps to get the file list for query:
|
||||
* 1、Load all files and partition values from the table path.
|
||||
* 2、Do the partition prune by the partition filter condition.
|
||||
*
|
||||
* There are 3 cases for this:
|
||||
* 1、If the partition columns size is equal to the actually partition path level, we
|
||||
* read it as partitioned table.(e.g partition column is "dt", the partition path is "2021-03-10")
|
||||
*
|
||||
* 2、If the partition columns size is not equal to the partition path level, but the partition
|
||||
* column size is "1" (e.g. partition column is "dt", but the partition path is "2021/03/10"
|
||||
* who's directory level is 3).We can still read it as a partitioned table. We will mapping the
|
||||
* partition path (e.g. 2021/03/10) to the only partition column (e.g. "dt").
|
||||
*
|
||||
* 3、Else the the partition columns size is not equal to the partition directory level and the
|
||||
* size is great than "1" (e.g. partition column is "dt,hh", the partition path is "2021/03/10/12")
|
||||
* , we read it as a Non-Partitioned table because we cannot know how to mapping the partition
|
||||
* path with the partition columns in this case.
|
||||
*
|
||||
*/
|
||||
case class HoodieFileIndex(
|
||||
spark: SparkSession,
|
||||
metaClient: HoodieTableMetaClient,
|
||||
schemaSpec: Option[StructType],
|
||||
options: Map[String, String],
|
||||
@transient fileStatusCache: FileStatusCache = NoopCache)
|
||||
extends FileIndex with Logging with SparkAdapterSupport {
|
||||
|
||||
private val basePath = metaClient.getBasePath
|
||||
|
||||
@transient private val queryPath = new Path(options.getOrElse("path", "'path' option required"))
|
||||
|
||||
private val queryType = options(QUERY_TYPE.key())
|
||||
|
||||
private val tableType = metaClient.getTableType
|
||||
|
||||
private val specifiedQueryInstant = options.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
|
||||
.map(HoodieSqlCommonUtils.formatQueryInstant)
|
||||
|
||||
/**
|
||||
* Get all completeCommits.
|
||||
*/
|
||||
lazy val completedCommits = metaClient.getCommitsTimeline
|
||||
.filterCompletedInstants().getInstants.iterator().toList.map(_.getTimestamp)
|
||||
|
||||
/**
|
||||
* Get the schema of the table.
|
||||
*/
|
||||
lazy val schema: StructType = schemaSpec.getOrElse({
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema)
|
||||
})
|
||||
|
||||
/**
|
||||
* Get the partition schema from the hoodie.properties.
|
||||
*/
|
||||
private lazy val _partitionSchemaFromProperties: StructType = {
|
||||
val tableConfig = metaClient.getTableConfig
|
||||
val partitionColumns = tableConfig.getPartitionFields
|
||||
val nameFieldMap = generateNameFieldMap(Right(schema))
|
||||
|
||||
if (partitionColumns.isPresent) {
|
||||
if (tableConfig.getKeyGeneratorClassName.equalsIgnoreCase(classOf[TimestampBasedKeyGenerator].getName)
|
||||
|| tableConfig.getKeyGeneratorClassName.equalsIgnoreCase(classOf[TimestampBasedAvroKeyGenerator].getName)) {
|
||||
val partitionFields = partitionColumns.get().map(column => StructField(column, StringType))
|
||||
StructType(partitionFields)
|
||||
} else {
|
||||
val partitionFields = partitionColumns.get().map(column =>
|
||||
nameFieldMap.getOrElse(column, throw new IllegalArgumentException(s"Cannot find column: '" +
|
||||
s"$column' in the schema[${schema.fields.mkString(",")}]")))
|
||||
StructType(partitionFields)
|
||||
}
|
||||
} else { // If the partition columns have not stored in hoodie.properties(the table that was
|
||||
// created earlier), we trait it as a non-partitioned table.
|
||||
logWarning("No partition columns available from hoodie.properties." +
|
||||
" Partition pruning will not work")
|
||||
new StructType()
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method traverses StructType recursively to build map of columnName -> StructField
|
||||
* Note : If there is nesting of columns like ["a.b.c.d", "a.b.c.e"] -> final map will have keys corresponding
|
||||
* only to ["a.b.c.d", "a.b.c.e"] and not for subsets like ["a.b.c", "a.b"]
|
||||
* @param structField
|
||||
* @return map of ( columns names -> StructField )
|
||||
*/
|
||||
private def generateNameFieldMap(structField: Either[StructField, StructType]) : Map[String, StructField] = {
|
||||
structField match {
|
||||
case Right(field) => field.fields.map(f => generateNameFieldMap(Left(f))).flatten.toMap
|
||||
case Left(field) => field.dataType match {
|
||||
case struct: StructType => generateNameFieldMap(Right(struct)).map {
|
||||
case (key: String, sf: StructField) => (field.name + "." + key, sf)
|
||||
}
|
||||
case _ => Map(field.name -> field)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private lazy val engineContext = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext))
|
||||
|
||||
private lazy val configProperties = {
|
||||
val sqlConf: SQLConf = spark.sessionState.conf
|
||||
val properties = new Properties()
|
||||
|
||||
// To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users
|
||||
// would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing.
|
||||
properties.setProperty(HoodieMetadataConfig.ENABLE.key(),
|
||||
sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
|
||||
properties.putAll(options.asJava)
|
||||
properties
|
||||
}
|
||||
|
||||
private lazy val fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
|
||||
.fromProperties(configProperties)
|
||||
.build()
|
||||
|
||||
private lazy val metadataConfig = HoodieMetadataConfig.newBuilder
|
||||
.fromProperties(configProperties)
|
||||
.build()
|
||||
|
||||
@transient @volatile private var fileSystemView: HoodieTableFileSystemView = _
|
||||
@transient @volatile private var cachedAllInputFileSlices: Map[PartitionRowPath, Seq[FileSlice]] = _
|
||||
@transient @volatile private var cachedFileSize: Long = 0L
|
||||
|
||||
@volatile private var queryAsNonePartitionedTable: Boolean = _
|
||||
|
||||
refresh0()
|
||||
|
||||
override def rootPaths: Seq[Path] = queryPath :: Nil
|
||||
|
||||
def enableDataSkipping(): Boolean = {
|
||||
options.getOrElse(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(),
|
||||
spark.sessionState.conf.getConfString(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "false")).toBoolean
|
||||
}
|
||||
|
||||
/**
|
||||
* Computes pruned list of candidate base-files' names based on provided list of {@link dataFilters}
|
||||
* conditions, by leveraging custom Column Statistics index (col-stats-index) bearing "min", "max",
|
||||
* "num_nulls" statistics for all clustered columns.
|
||||
*
|
||||
* NOTE: This method has to return complete set of candidate files, since only provided candidates will
|
||||
* ultimately be scanned as part of query execution. Hence, this method has to maintain the
|
||||
* invariant of conservatively including every base-file's name, that is NOT referenced in its index.
|
||||
*
|
||||
* @param queryFilters list of original data filters passed down from querying engine
|
||||
* @return list of pruned (data-skipped) candidate base-files' names
|
||||
*/
|
||||
private def lookupCandidateFilesInColStatsIndex(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try {
|
||||
val indexPath = metaClient.getColumnStatsIndexPath
|
||||
val fs = metaClient.getFs
|
||||
|
||||
if (!enableDataSkipping() || !fs.exists(new Path(indexPath)) || queryFilters.isEmpty) {
|
||||
// scalastyle:off return
|
||||
return Success(Option.empty)
|
||||
// scalastyle:on return
|
||||
}
|
||||
|
||||
// Collect all index tables present in `.zindex` folder
|
||||
val candidateIndexTables =
|
||||
fs.listStatus(new Path(indexPath))
|
||||
.filter(_.isDirectory)
|
||||
.map(_.getPath.getName)
|
||||
.filter(f => completedCommits.contains(f))
|
||||
.sortBy(x => x)
|
||||
|
||||
if (candidateIndexTables.isEmpty) {
|
||||
// scalastyle:off return
|
||||
return Success(Option.empty)
|
||||
// scalastyle:on return
|
||||
}
|
||||
|
||||
val dataFrameOpt = try {
|
||||
Some(spark.read.load(new Path(indexPath, candidateIndexTables.last).toString))
|
||||
} catch {
|
||||
case t: Throwable =>
|
||||
logError("Failed to read col-stats index; skipping", t)
|
||||
None
|
||||
}
|
||||
|
||||
dataFrameOpt.map(df => {
|
||||
val indexSchema = df.schema
|
||||
val indexFilter =
|
||||
queryFilters.map(createColumnStatsIndexFilterExpr(_, indexSchema))
|
||||
.reduce(And)
|
||||
|
||||
logInfo(s"Index filter condition: $indexFilter")
|
||||
|
||||
df.persist()
|
||||
|
||||
val allIndexedFileNames =
|
||||
df.select("file")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSet
|
||||
|
||||
val prunedCandidateFileNames =
|
||||
df.where(new Column(indexFilter))
|
||||
.select("file")
|
||||
.collect()
|
||||
.map(_.getString(0))
|
||||
.toSet
|
||||
|
||||
df.unpersist()
|
||||
|
||||
// NOTE: Col-Stats Index isn't guaranteed to have complete set of statistics for every
|
||||
// base-file: since it's bound to clustering, which could occur asynchronously
|
||||
// at arbitrary point in time, and is not likely to be touching all of the base files.
|
||||
//
|
||||
// To close that gap, we manually compute the difference b/w all indexed (by col-stats-index)
|
||||
// files and all outstanding base-files, and make sure that all base files not
|
||||
// represented w/in the index are included in the output of this method
|
||||
val notIndexedFileNames =
|
||||
lookupFileNamesMissingFromIndex(allIndexedFileNames)
|
||||
|
||||
prunedCandidateFileNames ++ notIndexedFileNames
|
||||
})
|
||||
}
|
||||
|
||||
private def lookupFileNamesMissingFromIndex(allIndexedFileNames: Set[String]) = {
|
||||
val allBaseFileNames = allFiles.map(f => f.getPath.getName).toSet
|
||||
allBaseFileNames -- allIndexedFileNames
|
||||
}
|
||||
|
||||
/**
|
||||
* Invoked by Spark to fetch list of latest base files per partition.
|
||||
*
|
||||
* @param partitionFilters partition column filters
|
||||
* @param dataFilters data columns filters
|
||||
* @return list of PartitionDirectory containing partition to base files mapping
|
||||
*/
|
||||
override def listFiles(partitionFilters: Seq[Expression],
|
||||
dataFilters: Seq[Expression]): Seq[PartitionDirectory] = {
|
||||
// Look up candidate files names in the col-stats index, if all of the following conditions are true
|
||||
// - Data-skipping is enabled
|
||||
// - Col-Stats Index is present
|
||||
// - List of predicates (filters) is present
|
||||
val candidateFilesNamesOpt: Option[Set[String]] =
|
||||
lookupCandidateFilesInColStatsIndex(dataFilters) match {
|
||||
case Success(opt) => opt
|
||||
case Failure(e) =>
|
||||
if (e.isInstanceOf[AnalysisException]) {
|
||||
logDebug("Failed to relay provided data filters to Z-index lookup", e)
|
||||
} else {
|
||||
logError("Failed to lookup candidate files in Z-index", e)
|
||||
}
|
||||
Option.empty
|
||||
}
|
||||
|
||||
logDebug(s"Overlapping candidate files (from Z-index): ${candidateFilesNamesOpt.getOrElse(Set.empty)}")
|
||||
|
||||
if (queryAsNonePartitionedTable) {
|
||||
// Read as Non-Partitioned table
|
||||
// Filter in candidate files based on the col-stats index lookup
|
||||
val candidateFiles =
|
||||
allFiles.filter(fileStatus =>
|
||||
// NOTE: This predicate is true when {@code Option} is empty
|
||||
candidateFilesNamesOpt.forall(_.contains(fileStatus.getPath.getName))
|
||||
)
|
||||
|
||||
logInfo(s"Total files : ${allFiles.size}; " +
|
||||
s"candidate files after data skipping: ${candidateFiles.size}; " +
|
||||
s"skipping percent ${if (allFiles.nonEmpty) (allFiles.size - candidateFiles.size) / allFiles.size.toDouble else 0}")
|
||||
|
||||
Seq(PartitionDirectory(InternalRow.empty, candidateFiles))
|
||||
} else {
|
||||
// Prune the partition path by the partition filters
|
||||
val prunedPartitions = prunePartition(cachedAllInputFileSlices.keys.toSeq, partitionFilters)
|
||||
var totalFileSize = 0
|
||||
var candidateFileSize = 0
|
||||
|
||||
val result = prunedPartitions.map { partition =>
|
||||
val baseFileStatuses: Seq[FileStatus] =
|
||||
cachedAllInputFileSlices(partition)
|
||||
.map(fs => fs.getBaseFile.orElse(null))
|
||||
.filter(_ != null)
|
||||
.map(_.getFileStatus)
|
||||
|
||||
// Filter in candidate files based on the col-stats index lookup
|
||||
val candidateFiles =
|
||||
baseFileStatuses.filter(fs =>
|
||||
// NOTE: This predicate is true when {@code Option} is empty
|
||||
candidateFilesNamesOpt.forall(_.contains(fs.getPath.getName)))
|
||||
|
||||
totalFileSize += baseFileStatuses.size
|
||||
candidateFileSize += candidateFiles.size
|
||||
PartitionDirectory(partition.values, candidateFiles)
|
||||
}
|
||||
|
||||
logInfo(s"Total base files: ${totalFileSize}; " +
|
||||
s"candidate files after data skipping : ${candidateFileSize}; " +
|
||||
s"skipping percent ${if (allFiles.nonEmpty) (totalFileSize - candidateFileSize) / totalFileSize.toDouble else 0}")
|
||||
|
||||
result
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch list of latest base files and log files per partition.
|
||||
*
|
||||
* @param partitionFilters partition column filters
|
||||
* @param dataFilters data column filters
|
||||
* @return mapping from string partition paths to its base/log files
|
||||
*/
|
||||
def listFileSlices(partitionFilters: Seq[Expression],
|
||||
dataFilters: Seq[Expression]): Map[String, Seq[FileSlice]] = {
|
||||
if (queryAsNonePartitionedTable) {
|
||||
// Read as Non-Partitioned table.
|
||||
cachedAllInputFileSlices.map(entry => (entry._1.partitionPath, entry._2))
|
||||
} else {
|
||||
// Prune the partition path by the partition filters
|
||||
val prunedPartitions = prunePartition(cachedAllInputFileSlices.keys.toSeq, partitionFilters)
|
||||
prunedPartitions.map(partition => {
|
||||
(partition.partitionPath, cachedAllInputFileSlices(partition))
|
||||
}).toMap
|
||||
}
|
||||
}
|
||||
|
||||
override def inputFiles: Array[String] = {
|
||||
val fileStatusList = allFiles
|
||||
fileStatusList.map(_.getPath.toString).toArray
|
||||
}
|
||||
|
||||
override def refresh(): Unit = {
|
||||
fileStatusCache.invalidateAll()
|
||||
refresh0()
|
||||
}
|
||||
|
||||
private def refresh0(): Unit = {
|
||||
val startTime = System.currentTimeMillis()
|
||||
val partitionFiles = loadPartitionPathFiles()
|
||||
val allFiles = partitionFiles.values.reduceOption(_ ++ _)
|
||||
.getOrElse(Array.empty[FileStatus])
|
||||
|
||||
metaClient.reloadActiveTimeline()
|
||||
val activeInstants = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
|
||||
val latestInstant = activeInstants.lastInstant()
|
||||
fileSystemView = new HoodieTableFileSystemView(metaClient, activeInstants, allFiles)
|
||||
val queryInstant = if (specifiedQueryInstant.isDefined) {
|
||||
specifiedQueryInstant
|
||||
} else if (latestInstant.isPresent) {
|
||||
Some(latestInstant.get.getTimestamp)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
(tableType, queryType) match {
|
||||
case (MERGE_ON_READ, QUERY_TYPE_SNAPSHOT_OPT_VAL) =>
|
||||
// Fetch and store latest base and log files, and their sizes
|
||||
cachedAllInputFileSlices = partitionFiles.map(p => {
|
||||
val latestSlices = if (latestInstant.isPresent) {
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p._1.partitionPath, queryInstant.get)
|
||||
.iterator().asScala.toSeq
|
||||
} else {
|
||||
Seq()
|
||||
}
|
||||
(p._1, latestSlices)
|
||||
})
|
||||
cachedFileSize = cachedAllInputFileSlices.values.flatten.map(fileSlice => {
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileLen + fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).sum
|
||||
} else {
|
||||
fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).sum
|
||||
}
|
||||
}).sum
|
||||
case (_, _) =>
|
||||
// Fetch and store latest base files and its sizes
|
||||
cachedAllInputFileSlices = partitionFiles.map(p => {
|
||||
val fileSlices = specifiedQueryInstant
|
||||
.map(instant =>
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(p._1.partitionPath, instant, true))
|
||||
.getOrElse(fileSystemView.getLatestFileSlices(p._1.partitionPath))
|
||||
.iterator().asScala.toSeq
|
||||
(p._1, fileSlices)
|
||||
})
|
||||
cachedFileSize = cachedAllInputFileSlices.values.flatten.map(fileSliceSize).sum
|
||||
}
|
||||
|
||||
// If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
|
||||
queryAsNonePartitionedTable = partitionFiles.keys.exists(p => p.values == InternalRow.empty)
|
||||
val flushSpend = System.currentTimeMillis() - startTime
|
||||
logInfo(s"Refresh table ${metaClient.getTableConfig.getTableName}," +
|
||||
s" spend: $flushSpend ms")
|
||||
}
|
||||
|
||||
private def fileSliceSize(fileSlice: FileSlice): Long = {
|
||||
val logFileSize = fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).filter(_ > 0).sum
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileLen + logFileSize
|
||||
} else {
|
||||
logFileSize
|
||||
}
|
||||
}
|
||||
|
||||
override def sizeInBytes: Long = {
|
||||
cachedFileSize
|
||||
}
|
||||
|
||||
override def partitionSchema: StructType = {
|
||||
if (queryAsNonePartitionedTable) {
|
||||
// If we read it as Non-Partitioned table, we should not
|
||||
// return the partition schema.
|
||||
new StructType()
|
||||
} else {
|
||||
_partitionSchemaFromProperties
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the data schema of the table.
|
||||
* @return
|
||||
*/
|
||||
def dataSchema: StructType = {
|
||||
val partitionColumns = partitionSchema.fields.map(_.name).toSet
|
||||
StructType(schema.fields.filterNot(f => partitionColumns.contains(f.name)))
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the FileStatus for all the base files (excluding log files). This should be used only for
|
||||
* cases where Spark directly fetches the list of files via HoodieFileIndex or for read optimized query logic
|
||||
* implemented internally within Hudi like HoodieBootstrapRelation. This helps avoid the use of path filter
|
||||
* to filter out log files within Spark.
|
||||
*
|
||||
* @return List of FileStatus for base files
|
||||
*/
|
||||
def allFiles: Seq[FileStatus] = {
|
||||
cachedAllInputFileSlices.values.flatten
|
||||
.filter(_.getBaseFile.isPresent)
|
||||
.map(_.getBaseFile.get().getFileStatus)
|
||||
.toSeq
|
||||
}
|
||||
|
||||
/**
|
||||
* Prune the partition by the filter.This implementation is fork from
|
||||
* org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex#prunePartitions.
|
||||
* @param partitionPaths All the partition paths.
|
||||
* @param predicates The filter condition.
|
||||
* @return The Pruned partition paths.
|
||||
*/
|
||||
def prunePartition(partitionPaths: Seq[PartitionRowPath],
|
||||
predicates: Seq[Expression]): Seq[PartitionRowPath] = {
|
||||
|
||||
val partitionColumnNames = partitionSchema.fields.map(_.name).toSet
|
||||
val partitionPruningPredicates = predicates.filter {
|
||||
_.references.map(_.name).toSet.subsetOf(partitionColumnNames)
|
||||
}
|
||||
if (partitionPruningPredicates.nonEmpty) {
|
||||
val predicate = partitionPruningPredicates.reduce(expressions.And)
|
||||
|
||||
val boundPredicate = InterpretedPredicate(predicate.transform {
|
||||
case a: AttributeReference =>
|
||||
val index = partitionSchema.indexWhere(a.name == _.name)
|
||||
BoundReference(index, partitionSchema(index).dataType, nullable = true)
|
||||
})
|
||||
|
||||
val prunedPartitionPaths = partitionPaths.filter {
|
||||
case PartitionRowPath(values, _) => boundPredicate.eval(values)
|
||||
}
|
||||
logInfo(s"Total partition size is: ${partitionPaths.size}," +
|
||||
s" after partition prune size is: ${prunedPartitionPaths.size}")
|
||||
prunedPartitionPaths
|
||||
} else {
|
||||
partitionPaths
|
||||
}
|
||||
}
|
||||
|
||||
def getAllQueryPartitionPaths: Seq[PartitionRowPath] = {
|
||||
val queryPartitionPath = FSUtils.getRelativePartitionPath(new Path(basePath), queryPath)
|
||||
// Load all the partition path from the basePath, and filter by the query partition path.
|
||||
// TODO load files from the queryPartitionPath directly.
|
||||
val partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath).asScala
|
||||
.filter(_.startsWith(queryPartitionPath))
|
||||
|
||||
val partitionSchema = _partitionSchemaFromProperties
|
||||
val timeZoneId = CaseInsensitiveMap(options)
|
||||
.get(DateTimeUtils.TIMEZONE_OPTION)
|
||||
.getOrElse(SQLConf.get.sessionLocalTimeZone)
|
||||
|
||||
val sparkParsePartitionUtil = sparkAdapter.createSparkParsePartitionUtil(spark
|
||||
.sessionState.conf)
|
||||
// Convert partition path to PartitionRowPath
|
||||
partitionPaths.map { partitionPath =>
|
||||
val partitionRow = if (partitionSchema.fields.length == 0) {
|
||||
// This is a non-partitioned table
|
||||
InternalRow.empty
|
||||
} else {
|
||||
val partitionFragments = partitionPath.split("/")
|
||||
|
||||
if (partitionFragments.length != partitionSchema.fields.length &&
|
||||
partitionSchema.fields.length == 1) {
|
||||
// If the partition column size is not equal to the partition fragment size
|
||||
// and the partition column size is 1, we map the whole partition path
|
||||
// to the partition column which can benefit from the partition prune.
|
||||
val prefix = s"${partitionSchema.fieldNames.head}="
|
||||
val partitionValue = if (partitionPath.startsWith(prefix)) {
|
||||
// support hive style partition path
|
||||
partitionPath.substring(prefix.length)
|
||||
} else {
|
||||
partitionPath
|
||||
}
|
||||
InternalRow.fromSeq(Seq(UTF8String.fromString(partitionValue)))
|
||||
} else if (partitionFragments.length != partitionSchema.fields.length &&
|
||||
partitionSchema.fields.length > 1) {
|
||||
// If the partition column size is not equal to the partition fragments size
|
||||
// and the partition column size > 1, we do not know how to map the partition
|
||||
// fragments to the partition columns. So we trait it as a Non-Partitioned Table
|
||||
// for the query which do not benefit from the partition prune.
|
||||
logWarning( s"Cannot do the partition prune for table $basePath." +
|
||||
s"The partitionFragments size (${partitionFragments.mkString(",")})" +
|
||||
s" is not equal to the partition columns size(${partitionSchema.fields.mkString(",")})")
|
||||
InternalRow.empty
|
||||
} else { // If partitionSeqs.length == partitionSchema.fields.length
|
||||
|
||||
// Append partition name to the partition value if the
|
||||
// HIVE_STYLE_PARTITIONING is disable.
|
||||
// e.g. convert "/xx/xx/2021/02" to "/xx/xx/year=2021/month=02"
|
||||
val partitionWithName =
|
||||
partitionFragments.zip(partitionSchema).map {
|
||||
case (partition, field) =>
|
||||
if (partition.indexOf("=") == -1) {
|
||||
s"${field.name}=$partition"
|
||||
} else {
|
||||
partition
|
||||
}
|
||||
}.mkString("/")
|
||||
val pathWithPartitionName = new Path(basePath, partitionWithName)
|
||||
val partitionDataTypes = partitionSchema.fields.map(f => f.name -> f.dataType).toMap
|
||||
|
||||
sparkParsePartitionUtil.parsePartition(pathWithPartitionName,
|
||||
typeInference = false, Set(new Path(basePath)), partitionDataTypes,
|
||||
DateTimeUtils.getTimeZone(timeZoneId))
|
||||
}
|
||||
}
|
||||
PartitionRowPath(partitionRow, partitionPath)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all partition paths and it's files under the query table path.
|
||||
*/
|
||||
private def loadPartitionPathFiles(): Map[PartitionRowPath, Array[FileStatus]] = {
|
||||
val partitionRowPaths = getAllQueryPartitionPaths
|
||||
// List files in all of the partition path.
|
||||
val pathToFetch = mutable.ArrayBuffer[PartitionRowPath]()
|
||||
val cachePartitionToFiles = mutable.Map[PartitionRowPath, Array[FileStatus]]()
|
||||
// Fetch from the FileStatusCache
|
||||
partitionRowPaths.foreach { partitionRowPath =>
|
||||
fileStatusCache.getLeafFiles(partitionRowPath.fullPartitionPath(basePath)) match {
|
||||
case Some(filesInPartition) =>
|
||||
cachePartitionToFiles.put(partitionRowPath, filesInPartition)
|
||||
|
||||
case None => pathToFetch.append(partitionRowPath)
|
||||
}
|
||||
}
|
||||
|
||||
val fetchedPartitionToFiles =
|
||||
if (pathToFetch.nonEmpty) {
|
||||
val fullPartitionPathsToFetch = pathToFetch.map(p => (p, p.fullPartitionPath(basePath).toString)).toMap
|
||||
val partitionToFilesMap = FSUtils.getFilesInPartitions(engineContext, metadataConfig, basePath,
|
||||
fullPartitionPathsToFetch.values.toArray, fileSystemStorageConfig.getSpillableDir)
|
||||
fullPartitionPathsToFetch.map(p => {
|
||||
(p._1, partitionToFilesMap.get(p._2))
|
||||
})
|
||||
} else {
|
||||
Map.empty[PartitionRowPath, Array[FileStatus]]
|
||||
}
|
||||
|
||||
// Update the fileStatusCache
|
||||
fetchedPartitionToFiles.foreach {
|
||||
case (partitionRowPath, filesInPartition) =>
|
||||
fileStatusCache.putLeafFiles(partitionRowPath.fullPartitionPath(basePath), filesInPartition)
|
||||
}
|
||||
cachePartitionToFiles.toMap ++ fetchedPartitionToFiles
|
||||
}
|
||||
|
||||
/**
|
||||
* Represent a partition path.
|
||||
* e.g. PartitionPath(InternalRow("2021","02","01"), "2021/02/01"))
|
||||
* @param values The partition values of this partition path.
|
||||
* @param partitionPath The partition path string.
|
||||
*/
|
||||
case class PartitionRowPath(values: InternalRow, partitionPath: String) {
|
||||
override def equals(other: Any): Boolean = other match {
|
||||
case PartitionRowPath(_, otherPath) => partitionPath == otherPath
|
||||
case _ => false
|
||||
}
|
||||
|
||||
override def hashCode(): Int = {
|
||||
partitionPath.hashCode
|
||||
}
|
||||
|
||||
def fullPartitionPath(basePath: String): Path = {
|
||||
if (partitionPath.isEmpty) {
|
||||
new Path(basePath) // This is a non-partition path
|
||||
} else {
|
||||
new Path(basePath, partitionPath)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,372 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.{GenericRecord, GenericRecordBuilder}
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner
|
||||
import org.apache.hudi.config.HoodiePayloadConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.HOODIE_RECORD_KEY_COL_POS
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.{HoodieAvroSerializer, HoodieAvroDeserializer}
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeProjection}
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch
|
||||
import org.apache.spark.{Partition, SerializableWritable, SparkContext, TaskContext}
|
||||
|
||||
import java.io.Closeable
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
import scala.util.Try
|
||||
|
||||
case class HoodieMergeOnReadPartition(index: Int, split: HoodieMergeOnReadFileSplit) extends Partition
|
||||
|
||||
class HoodieMergeOnReadRDD(@transient sc: SparkContext,
|
||||
@transient config: Configuration,
|
||||
fullSchemaFileReader: PartitionedFile => Iterator[Any],
|
||||
requiredSchemaFileReader: PartitionedFile => Iterator[Any],
|
||||
tableState: HoodieMergeOnReadTableState)
|
||||
extends RDD[InternalRow](sc, Nil) {
|
||||
|
||||
private val confBroadcast = sc.broadcast(new SerializableWritable(config))
|
||||
private val preCombineField = tableState.preCombineField
|
||||
private val recordKeyFieldOpt = tableState.recordKeyFieldOpt
|
||||
private val payloadProps = if (preCombineField.isDefined) {
|
||||
Some(HoodiePayloadConfig.newBuilder.withPayloadOrderingField(preCombineField.get).build.getProps)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = {
|
||||
val mergeOnReadPartition = split.asInstanceOf[HoodieMergeOnReadPartition]
|
||||
val iter = mergeOnReadPartition.split match {
|
||||
case dataFileOnlySplit if dataFileOnlySplit.logPaths.isEmpty =>
|
||||
read(dataFileOnlySplit.dataFile.get, requiredSchemaFileReader)
|
||||
case logFileOnlySplit if logFileOnlySplit.dataFile.isEmpty =>
|
||||
logFileIterator(logFileOnlySplit, getConfig)
|
||||
case skipMergeSplit if skipMergeSplit.mergeType
|
||||
.equals(DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL) =>
|
||||
skipMergeFileIterator(
|
||||
skipMergeSplit,
|
||||
read(skipMergeSplit.dataFile.get, requiredSchemaFileReader),
|
||||
getConfig
|
||||
)
|
||||
case payloadCombineSplit if payloadCombineSplit.mergeType
|
||||
.equals(DataSourceReadOptions.REALTIME_PAYLOAD_COMBINE_OPT_VAL) =>
|
||||
payloadCombineFileIterator(
|
||||
payloadCombineSplit,
|
||||
read(payloadCombineSplit.dataFile.get, fullSchemaFileReader),
|
||||
getConfig
|
||||
)
|
||||
case _ => throw new HoodieException(s"Unable to select an Iterator to read the Hoodie MOR File Split for " +
|
||||
s"file path: ${mergeOnReadPartition.split.dataFile.get.filePath}" +
|
||||
s"log paths: ${mergeOnReadPartition.split.logPaths.toString}" +
|
||||
s"hoodie table path: ${mergeOnReadPartition.split.tablePath}" +
|
||||
s"spark partition Index: ${mergeOnReadPartition.index}" +
|
||||
s"merge type: ${mergeOnReadPartition.split.mergeType}")
|
||||
}
|
||||
if (iter.isInstanceOf[Closeable]) {
|
||||
// register a callback to close logScanner which will be executed on task completion.
|
||||
// when tasks finished, this method will be called, and release resources.
|
||||
Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.asInstanceOf[Closeable].close()))
|
||||
}
|
||||
iter
|
||||
}
|
||||
|
||||
override protected def getPartitions: Array[Partition] = {
|
||||
tableState
|
||||
.hoodieRealtimeFileSplits
|
||||
.zipWithIndex
|
||||
.map(file => HoodieMergeOnReadPartition(file._2, file._1)).toArray
|
||||
}
|
||||
|
||||
private def getConfig: Configuration = {
|
||||
val conf = confBroadcast.value.value
|
||||
HoodieMergeOnReadRDD.CONFIG_INSTANTIATION_LOCK.synchronized {
|
||||
new Configuration(conf)
|
||||
}
|
||||
}
|
||||
|
||||
private def read(partitionedFile: PartitionedFile,
|
||||
readFileFunction: PartitionedFile => Iterator[Any]): Iterator[InternalRow] = {
|
||||
val fileIterator = readFileFunction(partitionedFile)
|
||||
val rows = fileIterator.flatMap(_ match {
|
||||
case r: InternalRow => Seq(r)
|
||||
case b: ColumnarBatch => b.rowIterator().asScala
|
||||
})
|
||||
rows
|
||||
}
|
||||
|
||||
private def logFileIterator(split: HoodieMergeOnReadFileSplit,
|
||||
config: Configuration): Iterator[InternalRow] =
|
||||
new Iterator[InternalRow] with Closeable {
|
||||
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
|
||||
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
|
||||
private val requiredFieldPosition =
|
||||
tableState.requiredStructSchema
|
||||
.map(f => tableAvroSchema.getField(f.name).pos()).toList
|
||||
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
|
||||
private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
|
||||
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
|
||||
private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config)
|
||||
private val logRecords = logScanner.getRecords
|
||||
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
|
||||
|
||||
private var recordToLoad: InternalRow = _
|
||||
override def hasNext: Boolean = {
|
||||
if (logRecordsKeyIterator.hasNext) {
|
||||
val curAvrokey = logRecordsKeyIterator.next()
|
||||
val curAvroRecord = logRecords.get(curAvrokey).getData.getInsertValue(tableAvroSchema)
|
||||
if (!curAvroRecord.isPresent) {
|
||||
// delete record found, skipping
|
||||
this.hasNext
|
||||
} else {
|
||||
val requiredAvroRecord = AvroConversionUtils
|
||||
.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder)
|
||||
recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
|
||||
true
|
||||
}
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
override def next(): InternalRow = {
|
||||
recordToLoad
|
||||
}
|
||||
|
||||
override def close(): Unit = {
|
||||
if (logScanner != null) {
|
||||
try {
|
||||
logScanner.close()
|
||||
} finally {
|
||||
logScanner = null
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def skipMergeFileIterator(split: HoodieMergeOnReadFileSplit,
|
||||
baseFileIterator: Iterator[InternalRow],
|
||||
config: Configuration): Iterator[InternalRow] =
|
||||
new Iterator[InternalRow] with Closeable {
|
||||
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
|
||||
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
|
||||
private val requiredFieldPosition =
|
||||
tableState.requiredStructSchema
|
||||
.map(f => tableAvroSchema.getField(f.name).pos()).toList
|
||||
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
|
||||
private val deserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
|
||||
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
|
||||
private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config)
|
||||
private val logRecords = logScanner.getRecords
|
||||
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
|
||||
|
||||
private var recordToLoad: InternalRow = _
|
||||
|
||||
@scala.annotation.tailrec
|
||||
override def hasNext: Boolean = {
|
||||
if (baseFileIterator.hasNext) {
|
||||
recordToLoad = baseFileIterator.next()
|
||||
true
|
||||
} else {
|
||||
if (logRecordsKeyIterator.hasNext) {
|
||||
val curAvrokey = logRecordsKeyIterator.next()
|
||||
val curAvroRecord = logRecords.get(curAvrokey).getData.getInsertValue(tableAvroSchema)
|
||||
if (!curAvroRecord.isPresent) {
|
||||
// delete record found, skipping
|
||||
this.hasNext
|
||||
} else {
|
||||
val requiredAvroRecord = AvroConversionUtils
|
||||
.buildAvroRecordBySchema(curAvroRecord.get(), requiredAvroSchema, requiredFieldPosition, recordBuilder)
|
||||
recordToLoad = unsafeProjection(deserializer.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
|
||||
true
|
||||
}
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def next(): InternalRow = {
|
||||
recordToLoad
|
||||
}
|
||||
|
||||
override def close(): Unit = {
|
||||
if (logScanner != null) {
|
||||
try {
|
||||
logScanner.close()
|
||||
} finally {
|
||||
logScanner = null
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def payloadCombineFileIterator(split: HoodieMergeOnReadFileSplit,
|
||||
baseFileIterator: Iterator[InternalRow],
|
||||
config: Configuration): Iterator[InternalRow] =
|
||||
new Iterator[InternalRow] with Closeable {
|
||||
private val tableAvroSchema = new Schema.Parser().parse(tableState.tableAvroSchema)
|
||||
private val requiredAvroSchema = new Schema.Parser().parse(tableState.requiredAvroSchema)
|
||||
private val requiredFieldPosition =
|
||||
tableState.requiredStructSchema
|
||||
.map(f => tableAvroSchema.getField(f.name).pos()).toList
|
||||
private val serializer = HoodieAvroSerializer(tableState.tableStructSchema, tableAvroSchema, false)
|
||||
private val requiredDeserializer = HoodieAvroDeserializer(requiredAvroSchema, tableState.requiredStructSchema)
|
||||
private val recordBuilder = new GenericRecordBuilder(requiredAvroSchema)
|
||||
private val unsafeProjection = UnsafeProjection.create(tableState.requiredStructSchema)
|
||||
private var logScanner = HoodieMergeOnReadRDD.scanLog(split, tableAvroSchema, config)
|
||||
private val logRecords = logScanner.getRecords
|
||||
private val logRecordsKeyIterator = logRecords.keySet().iterator().asScala
|
||||
private val keyToSkip = mutable.Set.empty[String]
|
||||
private val recordKeyPosition = if (recordKeyFieldOpt.isEmpty) HOODIE_RECORD_KEY_COL_POS else tableState.tableStructSchema.fieldIndex(recordKeyFieldOpt.get)
|
||||
|
||||
private var recordToLoad: InternalRow = _
|
||||
|
||||
@scala.annotation.tailrec
|
||||
override def hasNext: Boolean = {
|
||||
if (baseFileIterator.hasNext) {
|
||||
val curRow = baseFileIterator.next()
|
||||
val curKey = curRow.getString(recordKeyPosition)
|
||||
if (logRecords.containsKey(curKey)) {
|
||||
// duplicate key found, merging
|
||||
keyToSkip.add(curKey)
|
||||
val mergedAvroRecord = mergeRowWithLog(curRow, curKey)
|
||||
if (!mergedAvroRecord.isPresent) {
|
||||
// deleted
|
||||
this.hasNext
|
||||
} else {
|
||||
// load merged record as InternalRow with required schema
|
||||
val requiredAvroRecord = AvroConversionUtils
|
||||
.buildAvroRecordBySchema(
|
||||
mergedAvroRecord.get(),
|
||||
requiredAvroSchema,
|
||||
requiredFieldPosition,
|
||||
recordBuilder
|
||||
)
|
||||
recordToLoad = unsafeProjection(requiredDeserializer
|
||||
.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
|
||||
true
|
||||
}
|
||||
} else {
|
||||
// No merge needed, load current row with required schema
|
||||
recordToLoad = unsafeProjection(createRowWithRequiredSchema(curRow))
|
||||
true
|
||||
}
|
||||
} else {
|
||||
if (logRecordsKeyIterator.hasNext) {
|
||||
val curKey = logRecordsKeyIterator.next()
|
||||
if (keyToSkip.contains(curKey)) {
|
||||
this.hasNext
|
||||
} else {
|
||||
val insertAvroRecord =
|
||||
logRecords.get(curKey).getData.getInsertValue(tableAvroSchema)
|
||||
if (!insertAvroRecord.isPresent) {
|
||||
// stand alone delete record, skipping
|
||||
this.hasNext
|
||||
} else {
|
||||
val requiredAvroRecord = AvroConversionUtils
|
||||
.buildAvroRecordBySchema(
|
||||
insertAvroRecord.get(),
|
||||
requiredAvroSchema,
|
||||
requiredFieldPosition,
|
||||
recordBuilder
|
||||
)
|
||||
recordToLoad = unsafeProjection(requiredDeserializer
|
||||
.deserializeData(requiredAvroRecord).asInstanceOf[InternalRow])
|
||||
true
|
||||
}
|
||||
}
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override def next(): InternalRow = recordToLoad
|
||||
|
||||
override def close(): Unit = {
|
||||
if (logScanner != null) {
|
||||
try {
|
||||
logScanner.close()
|
||||
} finally {
|
||||
logScanner = null
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def createRowWithRequiredSchema(row: InternalRow): InternalRow = {
|
||||
val rowToReturn = new SpecificInternalRow(tableState.requiredStructSchema)
|
||||
val posIterator = requiredFieldPosition.iterator
|
||||
var curIndex = 0
|
||||
tableState.requiredStructSchema.foreach(
|
||||
f => {
|
||||
val curPos = posIterator.next()
|
||||
val curField = if (row.isNullAt(curPos)) null else row.get(curPos, f.dataType)
|
||||
rowToReturn.update(curIndex, curField)
|
||||
curIndex = curIndex + 1
|
||||
}
|
||||
)
|
||||
rowToReturn
|
||||
}
|
||||
|
||||
private def mergeRowWithLog(curRow: InternalRow, curKey: String) = {
|
||||
val historyAvroRecord = serializer.serialize(curRow).asInstanceOf[GenericRecord]
|
||||
if (payloadProps.isDefined) {
|
||||
logRecords.get(curKey).getData.combineAndGetUpdateValue(historyAvroRecord,
|
||||
tableAvroSchema, payloadProps.get)
|
||||
} else {
|
||||
logRecords.get(curKey).getData.combineAndGetUpdateValue(historyAvroRecord, tableAvroSchema)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private object HoodieMergeOnReadRDD {
|
||||
val CONFIG_INSTANTIATION_LOCK = new Object()
|
||||
|
||||
def scanLog(split: HoodieMergeOnReadFileSplit, logSchema: Schema, config: Configuration): HoodieMergedLogRecordScanner = {
|
||||
val fs = FSUtils.getFs(split.tablePath, config)
|
||||
HoodieMergedLogRecordScanner.newBuilder()
|
||||
.withFileSystem(fs)
|
||||
.withBasePath(split.tablePath)
|
||||
.withLogFilePaths(split.logPaths.get.asJava)
|
||||
.withReaderSchema(logSchema)
|
||||
.withLatestInstantTime(split.latestCommit)
|
||||
.withReadBlocksLazily(
|
||||
Try(config.get(HoodieRealtimeConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED).toBoolean)
|
||||
.getOrElse(false))
|
||||
.withReverseReader(false)
|
||||
.withBufferSize(
|
||||
config.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
|
||||
.withMaxMemorySizeInBytes(split.maxCompactionMemoryInBytes)
|
||||
.withSpillableMapBasePath(
|
||||
config.get(HoodieRealtimeConfig.SPILLABLE_MAP_BASE_PATH_PROP,
|
||||
HoodieRealtimeConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH))
|
||||
.build()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,735 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hadoop.hive.conf.HiveConf
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.HoodieWriterUtils._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient}
|
||||
import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, HoodieTimelineTimeZone, WriteOperationType}
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME}
|
||||
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows}
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||
import org.apache.hudi.table.BulkInsertPartitioner
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession}
|
||||
import org.apache.spark.SparkContext
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
object HoodieSparkSqlWriter {
|
||||
|
||||
private val log = LogManager.getLogger(getClass)
|
||||
private var tableExists: Boolean = false
|
||||
private var asyncCompactionTriggerFnDefined: Boolean = false
|
||||
private var asyncClusteringTriggerFnDefined: Boolean = false
|
||||
|
||||
def write(sqlContext: SQLContext,
|
||||
mode: SaveMode,
|
||||
optParams: Map[String, String],
|
||||
df: DataFrame,
|
||||
hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
|
||||
hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
|
||||
asyncCompactionTriggerFn: Option[Function1[SparkRDDWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty,
|
||||
asyncClusteringTriggerFn: Option[Function1[SparkRDDWriteClient[HoodieRecordPayload[Nothing]], Unit]] = Option.empty
|
||||
)
|
||||
: (Boolean, common.util.Option[String], common.util.Option[String], common.util.Option[String],
|
||||
SparkRDDWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
|
||||
|
||||
assert(optParams.get("path").exists(!StringUtils.isNullOrEmpty(_)), "'path' must be set")
|
||||
val path = optParams("path")
|
||||
val basePath = new Path(path)
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
|
||||
tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
|
||||
var tableConfig = getHoodieTableConfig(sparkContext, path, hoodieTableConfigOpt)
|
||||
validateTableConfig(sqlContext.sparkSession, optParams, tableConfig)
|
||||
|
||||
val (parameters, hoodieConfig) = mergeParamsAndGetHoodieConfig(optParams, tableConfig)
|
||||
val tblName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME,
|
||||
s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.").trim
|
||||
assert(!StringUtils.isNullOrEmpty(hoodieConfig.getString(HoodieWriteConfig.TBL_NAME)),
|
||||
s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
|
||||
|
||||
asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined
|
||||
asyncClusteringTriggerFnDefined = asyncClusteringTriggerFn.isDefined
|
||||
sparkContext.getConf.getOption("spark.serializer") match {
|
||||
case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") =>
|
||||
case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer")
|
||||
}
|
||||
val tableType = HoodieTableType.valueOf(hoodieConfig.getString(TABLE_TYPE))
|
||||
var operation = WriteOperationType.fromValue(hoodieConfig.getString(OPERATION))
|
||||
// It does not make sense to allow upsert() operation if INSERT_DROP_DUPS is true
|
||||
// Auto-correct the operation to "insert" if OPERATION is set to "upsert" wrongly
|
||||
// or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) .
|
||||
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS) &&
|
||||
operation == WriteOperationType.UPSERT) {
|
||||
|
||||
log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " +
|
||||
s"when $INSERT_DROP_DUPS is set to be true, " +
|
||||
s"overriding the $OPERATION to be $INSERT_OPERATION_OPT_VAL")
|
||||
|
||||
operation = WriteOperationType.INSERT
|
||||
}
|
||||
|
||||
val jsc = new JavaSparkContext(sparkContext)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime()
|
||||
val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(hoodieConfig.getProps))
|
||||
|
||||
if (mode == SaveMode.Ignore && tableExists) {
|
||||
log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.")
|
||||
(false, common.util.Option.empty(), common.util.Option.empty(), common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig)
|
||||
} else {
|
||||
// Handle various save modes
|
||||
handleSaveModes(sqlContext.sparkSession, mode, basePath, tableConfig, tblName, operation, fs)
|
||||
val partitionColumns = HoodieSparkUtils.getPartitionColumns(keyGenerator, toProperties(parameters))
|
||||
// Create the table if not present
|
||||
if (!tableExists) {
|
||||
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT)
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
|
||||
val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
|
||||
val populateMetaFields = hoodieConfig.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS)
|
||||
|
||||
val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(tableType)
|
||||
.setTableName(tblName)
|
||||
.setRecordKeyFields(recordKeyFields)
|
||||
.setBaseFileFormat(baseFileFormat)
|
||||
.setArchiveLogFolder(archiveLogFolder)
|
||||
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
|
||||
.setPartitionFields(partitionColumns)
|
||||
.setPopulateMetaFields(populateMetaFields)
|
||||
.setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
|
||||
.setKeyGeneratorClassProp(HoodieWriterUtils.getOriginKeyGenerator(parameters))
|
||||
.setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING))
|
||||
.setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING))
|
||||
.setCommitTimezone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getStringOrDefault(HoodieTableConfig.TIMELINE_TIMEZONE)))
|
||||
.initTable(sparkContext.hadoopConfiguration, path)
|
||||
tableConfig = tableMetaClient.getTableConfig
|
||||
}
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(operation, tableConfig.getTableType)
|
||||
val dropPartitionColumns = hoodieConfig.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS)
|
||||
|
||||
// short-circuit if bulk_insert via row is enabled.
|
||||
// scalastyle:off
|
||||
if (hoodieConfig.getBoolean(ENABLE_ROW_WRITER) &&
|
||||
operation == WriteOperationType.BULK_INSERT) {
|
||||
val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName,
|
||||
basePath, path, instantTime, partitionColumns)
|
||||
return (success, commitTime, common.util.Option.empty(), common.util.Option.empty(), hoodieWriteClient.orNull, tableConfig)
|
||||
}
|
||||
// scalastyle:on
|
||||
|
||||
val reconcileSchema = parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean
|
||||
val (writeResult, writeClient: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]) =
|
||||
operation match {
|
||||
case WriteOperationType.DELETE => {
|
||||
val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema)
|
||||
// Convert to RDD[HoodieKey]
|
||||
val hoodieKeysToDelete = genericRecords.map(gr => keyGenerator.getKey(gr)).toJavaRDD()
|
||||
|
||||
if (!tableExists) {
|
||||
throw new HoodieException(s"hoodie table at $basePath does not exist")
|
||||
}
|
||||
|
||||
// Create a HoodieWriteClient & issue the delete.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
null, path, tblName,
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
asyncCompactionTriggerFn.get.apply(client)
|
||||
}
|
||||
if (isAsyncClusteringEnabled(client, parameters)) {
|
||||
asyncClusteringTriggerFn.get.apply(client)
|
||||
}
|
||||
|
||||
// Issue deletes
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
val writeStatuses = DataSourceUtils.doDeleteOperation(client, hoodieKeysToDelete, instantTime)
|
||||
(writeStatuses, client)
|
||||
}
|
||||
case WriteOperationType.DELETE_PARTITION => {
|
||||
val genericRecords = registerKryoClassesAndGetGenericRecords(tblName, sparkContext, df, reconcileSchema)
|
||||
if (!tableExists) {
|
||||
throw new HoodieException(s"hoodie table at $basePath does not exist")
|
||||
}
|
||||
|
||||
// Get list of partitions to delete
|
||||
val partitionsToDelete = if (parameters.containsKey(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key())) {
|
||||
val partitionColsToDelete = parameters.get(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key()).get.split(",")
|
||||
java.util.Arrays.asList(partitionColsToDelete:_*)
|
||||
} else {
|
||||
genericRecords.map(gr => keyGenerator.getKey(gr).getPartitionPath).toJavaRDD().distinct().collect()
|
||||
}
|
||||
// Create a HoodieWriteClient & issue the delete.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
null, path, tblName,
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
// Issue delete partitions
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
val writeStatuses = DataSourceUtils.doDeletePartitionsOperation(client, partitionsToDelete, instantTime)
|
||||
(writeStatuses, client)
|
||||
}
|
||||
case _ => { // any other operation
|
||||
// register classes & schemas
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
|
||||
sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
if (reconcileSchema) {
|
||||
schema = getLatestTableSchema(fs, basePath, sparkContext, schema)
|
||||
}
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
log.info(s"Registered avro schema : ${schema.toString(true)}")
|
||||
|
||||
// Convert to RDD[HoodieRecord]
|
||||
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, structName, nameSpace, reconcileSchema,
|
||||
org.apache.hudi.common.util.Option.of(schema))
|
||||
val shouldCombine = parameters(INSERT_DROP_DUPS.key()).toBoolean ||
|
||||
operation.equals(WriteOperationType.UPSERT) ||
|
||||
parameters.getOrElse(HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(),
|
||||
HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue()).toBoolean
|
||||
val hoodieAllIncomingRecords = genericRecords.map(gr => {
|
||||
val processedRecord = getProcessedRecord(partitionColumns, gr, dropPartitionColumns)
|
||||
val hoodieRecord = if (shouldCombine) {
|
||||
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, hoodieConfig.getString(PRECOMBINE_FIELD), false, parameters.getOrElse(
|
||||
DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
|
||||
DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean)
|
||||
.asInstanceOf[Comparable[_]]
|
||||
DataSourceUtils.createHoodieRecord(processedRecord,
|
||||
orderingVal, keyGenerator.getKey(gr),
|
||||
hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||
} else {
|
||||
DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||
}
|
||||
hoodieRecord
|
||||
}).toJavaRDD()
|
||||
|
||||
val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path,
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
|
||||
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
asyncCompactionTriggerFn.get.apply(client)
|
||||
}
|
||||
|
||||
if (isAsyncClusteringEnabled(client, parameters)) {
|
||||
asyncClusteringTriggerFn.get.apply(client)
|
||||
}
|
||||
|
||||
val hoodieRecords =
|
||||
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS)) {
|
||||
DataSourceUtils.dropDuplicates(jsc, hoodieAllIncomingRecords, mapAsJavaMap(parameters))
|
||||
} else {
|
||||
hoodieAllIncomingRecords
|
||||
}
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
val writeResult = DataSourceUtils.doWriteOperation(client, hoodieRecords, instantTime, operation)
|
||||
(writeResult, client)
|
||||
}
|
||||
}
|
||||
|
||||
// Check for errors and commit the write.
|
||||
val (writeSuccessful, compactionInstant, clusteringInstant) =
|
||||
commitAndPerformPostOperations(sqlContext.sparkSession, df.schema,
|
||||
writeResult, parameters, writeClient, tableConfig, jsc,
|
||||
TableInstantInfo(basePath, instantTime, commitActionType, operation))
|
||||
|
||||
(writeSuccessful, common.util.Option.ofNullable(instantTime), compactionInstant, clusteringInstant, writeClient, tableConfig)
|
||||
}
|
||||
}
|
||||
|
||||
def generateSchemaWithoutPartitionColumns(partitionParam: String, schema: Schema): Schema = {
|
||||
val fieldsToRemove = new java.util.ArrayList[String]()
|
||||
partitionParam.split(",").map(partitionField => partitionField.trim)
|
||||
.filter(s => !s.isEmpty).map(field => fieldsToRemove.add(field))
|
||||
HoodieAvroUtils.removeFields(schema, fieldsToRemove)
|
||||
}
|
||||
|
||||
def getProcessedRecord(partitionParam: String, record: GenericRecord,
|
||||
dropPartitionColumns: Boolean): GenericRecord = {
|
||||
var processedRecord = record
|
||||
if (dropPartitionColumns) {
|
||||
val writeSchema = generateSchemaWithoutPartitionColumns(partitionParam, record.getSchema)
|
||||
processedRecord = HoodieAvroUtils.rewriteRecord(record, writeSchema)
|
||||
}
|
||||
processedRecord
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if schema needs upgrade (if incoming record's write schema is old while table schema got evolved).
|
||||
*
|
||||
* @param fs instance of FileSystem.
|
||||
* @param basePath base path.
|
||||
* @param sparkContext instance of spark context.
|
||||
* @param schema incoming record's schema.
|
||||
* @return Pair of(boolean, table schema), where first entry will be true only if schema conversion is required.
|
||||
*/
|
||||
def getLatestTableSchema(fs: FileSystem, basePath: Path, sparkContext: SparkContext, schema: Schema): Schema = {
|
||||
var latestSchema: Schema = schema
|
||||
if (FSUtils.isTableExists(basePath.toString, fs)) {
|
||||
val tableMetaClient = HoodieTableMetaClient.builder.setConf(sparkContext.hadoopConfiguration).setBasePath(basePath.toString).build()
|
||||
val tableSchemaResolver = new TableSchemaResolver(tableMetaClient)
|
||||
latestSchema = tableSchemaResolver.getLatestSchema(schema, false, null);
|
||||
}
|
||||
latestSchema
|
||||
}
|
||||
|
||||
def registerKryoClassesAndGetGenericRecords(tblName: String, sparkContext : SparkContext, df: Dataset[Row],
|
||||
reconcileSchema: Boolean) : RDD[GenericRecord] = {
|
||||
val structName = s"${tblName}_record"
|
||||
val nameSpace = s"hoodie.${tblName}"
|
||||
sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
HoodieSparkUtils.createRdd(df, structName, nameSpace, reconcileSchema)
|
||||
}
|
||||
|
||||
def bootstrap(sqlContext: SQLContext,
|
||||
mode: SaveMode,
|
||||
optParams: Map[String, String],
|
||||
df: DataFrame,
|
||||
hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
|
||||
hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty): Boolean = {
|
||||
|
||||
assert(optParams.get("path").exists(!StringUtils.isNullOrEmpty(_)), "'path' must be set")
|
||||
val path = optParams("path")
|
||||
val basePath = new Path(path)
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
|
||||
tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
|
||||
var tableConfig = getHoodieTableConfig(sparkContext, path, hoodieTableConfigOpt)
|
||||
validateTableConfig(sqlContext.sparkSession, optParams, tableConfig)
|
||||
|
||||
val (parameters, hoodieConfig) = mergeParamsAndGetHoodieConfig(optParams, tableConfig)
|
||||
val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
|
||||
val tableType = hoodieConfig.getStringOrDefault(TABLE_TYPE)
|
||||
val bootstrapBasePath = hoodieConfig.getStringOrThrow(BASE_PATH,
|
||||
s"'${BASE_PATH.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
|
||||
" operation'")
|
||||
val bootstrapIndexClass = hoodieConfig.getStringOrDefault(INDEX_CLASS_NAME)
|
||||
|
||||
var schema: String = null
|
||||
if (df.schema.nonEmpty) {
|
||||
val (structName, namespace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableName)
|
||||
schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, namespace).toString
|
||||
} else {
|
||||
schema = HoodieAvroUtils.getNullSchema.toString
|
||||
}
|
||||
|
||||
|
||||
// Handle various save modes
|
||||
if (mode == SaveMode.Ignore && tableExists) {
|
||||
log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.")
|
||||
false
|
||||
} else {
|
||||
handleSaveModes(sqlContext.sparkSession, mode, basePath, tableConfig, tableName, WriteOperationType.BOOTSTRAP, fs)
|
||||
}
|
||||
|
||||
if (!tableExists) {
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
|
||||
val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters)
|
||||
val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
|
||||
val keyGenProp = hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT)
|
||||
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.valueOf(tableType))
|
||||
.setTableName(tableName)
|
||||
.setRecordKeyFields(recordKeyFields)
|
||||
.setArchiveLogFolder(archiveLogFolder)
|
||||
.setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS_NAME))
|
||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
|
||||
.setBootstrapIndexClass(bootstrapIndexClass)
|
||||
.setBaseFileFormat(baseFileFormat)
|
||||
.setBootstrapBasePath(bootstrapBasePath)
|
||||
.setPartitionFields(partitionColumns)
|
||||
.setPopulateMetaFields(populateMetaFields)
|
||||
.setKeyGeneratorClassProp(keyGenProp)
|
||||
.setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING))
|
||||
.setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING))
|
||||
.setCommitTimezone(HoodieTimelineTimeZone.valueOf(hoodieConfig.getStringOrDefault(HoodieTableConfig.TIMELINE_TIMEZONE)))
|
||||
.initTable(sparkContext.hadoopConfiguration, path)
|
||||
}
|
||||
|
||||
val jsc = new JavaSparkContext(sqlContext.sparkContext)
|
||||
val writeClient = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
schema, path, tableName, mapAsJavaMap(parameters)))
|
||||
try {
|
||||
writeClient.bootstrap(org.apache.hudi.common.util.Option.empty())
|
||||
} finally {
|
||||
writeClient.close()
|
||||
}
|
||||
val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
|
||||
metaSyncSuccess
|
||||
}
|
||||
|
||||
def bulkInsertAsRow(sqlContext: SQLContext,
|
||||
parameters: Map[String, String],
|
||||
df: DataFrame,
|
||||
tblName: String,
|
||||
basePath: Path,
|
||||
path: String,
|
||||
instantTime: String,
|
||||
partitionColumns: String): (Boolean, common.util.Option[String]) = {
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
|
||||
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
val dropPartitionColumns =
|
||||
parameters.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key(), DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()).toBoolean
|
||||
// register classes & schemas
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
|
||||
sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
var schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
if (dropPartitionColumns) {
|
||||
schema = generateSchemaWithoutPartitionColumns(partitionColumns, schema)
|
||||
}
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
log.info(s"Registered avro schema : ${schema.toString(true)}")
|
||||
if (parameters(INSERT_DROP_DUPS.key).toBoolean) {
|
||||
throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
|
||||
}
|
||||
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA_STRING.key, schema.toString)
|
||||
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path, tblName, mapAsJavaMap(params))
|
||||
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
|
||||
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
|
||||
if (userDefinedBulkInsertPartitionerOpt.isPresent) {
|
||||
userDefinedBulkInsertPartitionerOpt.get
|
||||
}
|
||||
else {
|
||||
BulkInsertInternalPartitionerWithRowsFactory.get(writeConfig.getBulkInsertSortMode)
|
||||
}
|
||||
} else {
|
||||
// Sort modes are not yet supported when meta fields are disabled
|
||||
new NonSortPartitionerWithRows()
|
||||
}
|
||||
val arePartitionRecordsSorted = bulkInsertPartitionerRows.arePartitionRecordsSorted();
|
||||
parameters.updated(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED, arePartitionRecordsSorted.toString)
|
||||
val isGlobalIndex = if (populateMetaFields) {
|
||||
SparkHoodieIndexFactory.isGlobalIndex(writeConfig)
|
||||
} else {
|
||||
false
|
||||
}
|
||||
val hoodieDF = if (populateMetaFields) {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace,
|
||||
bulkInsertPartitionerRows, isGlobalIndex, dropPartitionColumns)
|
||||
} else {
|
||||
HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsertWithoutMetaFields(df)
|
||||
}
|
||||
if (HoodieSparkUtils.isSpark2) {
|
||||
hoodieDF.write.format("org.apache.hudi.internal")
|
||||
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
|
||||
.options(params)
|
||||
.mode(SaveMode.Append)
|
||||
.save()
|
||||
} else if(HoodieSparkUtils.isSpark3) {
|
||||
hoodieDF.write.format("org.apache.hudi.spark3.internal")
|
||||
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
|
||||
.option(HoodieInternalConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key, hoodieDF.schema.toDDL)
|
||||
.options(params)
|
||||
.mode(SaveMode.Append)
|
||||
.save()
|
||||
} else {
|
||||
throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
|
||||
+ " To use row writer please switch to spark 2 or spark 3")
|
||||
}
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params)
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
|
||||
val metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
|
||||
val syncHiveSuccess =
|
||||
if (hiveSyncEnabled || metaSyncEnabled) {
|
||||
metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
|
||||
} else {
|
||||
true
|
||||
}
|
||||
(syncHiveSuccess, common.util.Option.ofNullable(instantTime))
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
params.foreach(kv => props.setProperty(kv._1, kv._2))
|
||||
props
|
||||
}
|
||||
|
||||
private def handleSaveModes(spark: SparkSession, mode: SaveMode, tablePath: Path, tableConfig: HoodieTableConfig, tableName: String,
|
||||
operation: WriteOperationType, fs: FileSystem): Unit = {
|
||||
if (mode == SaveMode.Append && tableExists) {
|
||||
val existingTableName = tableConfig.getTableName
|
||||
val resolver = spark.sessionState.conf.resolver
|
||||
if (!resolver(existingTableName, tableName)) {
|
||||
throw new HoodieException(s"hoodie table with name $existingTableName already exists at $tablePath," +
|
||||
s" can not append data to the table with another name $tableName.")
|
||||
}
|
||||
}
|
||||
|
||||
if (operation != WriteOperationType.DELETE) {
|
||||
if (mode == SaveMode.ErrorIfExists && tableExists) {
|
||||
throw new HoodieException(s"hoodie table at $tablePath already exists.")
|
||||
} else if (mode == SaveMode.Overwrite && tableExists && operation != WriteOperationType.INSERT_OVERWRITE_TABLE) {
|
||||
// When user set operation as INSERT_OVERWRITE_TABLE,
|
||||
// overwrite will use INSERT_OVERWRITE_TABLE operator in doWriteOperation
|
||||
log.warn(s"hoodie table at $tablePath already exists. Deleting existing data & overwriting with new data.")
|
||||
fs.delete(tablePath, true)
|
||||
tableExists = false
|
||||
}
|
||||
} else {
|
||||
// Delete Operation only supports Append mode
|
||||
if (mode != SaveMode.Append) {
|
||||
throw new HoodieException(s"Append is the only save mode applicable for ${operation.toString} operation")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def syncHive(basePath: Path, fs: FileSystem, hoodieConfig: HoodieConfig, sqlConf: SQLConf): Boolean = {
|
||||
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, hoodieConfig, sqlConf)
|
||||
val hiveConf: HiveConf = new HiveConf()
|
||||
hiveConf.addResource(fs.getConf)
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable()
|
||||
true
|
||||
}
|
||||
|
||||
private def buildSyncConfig(basePath: Path, hoodieConfig: HoodieConfig, sqlConf: SQLConf): HiveSyncConfig = {
|
||||
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
|
||||
hiveSyncConfig.basePath = basePath.toString
|
||||
hiveSyncConfig.baseFileFormat = hoodieConfig.getString(HIVE_BASE_FILE_FORMAT)
|
||||
hiveSyncConfig.usePreApacheInputFormat =
|
||||
hoodieConfig.getStringOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT).toBoolean
|
||||
hiveSyncConfig.databaseName = hoodieConfig.getString(HIVE_DATABASE)
|
||||
hiveSyncConfig.tableName = hoodieConfig.getString(HIVE_TABLE)
|
||||
hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER)
|
||||
hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS)
|
||||
hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL)
|
||||
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE,
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean
|
||||
hiveSyncConfig.partitionFields =
|
||||
ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
|
||||
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS)
|
||||
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC)
|
||||
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)
|
||||
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean
|
||||
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean
|
||||
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean
|
||||
hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING).toBoolean
|
||||
hiveSyncConfig.batchSyncNum = hoodieConfig.getStringOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM).toInt
|
||||
|
||||
hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
hiveSyncConfig.sparkSchemaLengthThreshold = sqlConf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD)
|
||||
hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)
|
||||
hiveSyncConfig.syncMode = hoodieConfig.getString(HIVE_SYNC_MODE)
|
||||
hiveSyncConfig.serdeProperties = hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES)
|
||||
hiveSyncConfig.tableProperties = hoodieConfig.getString(HIVE_TABLE_PROPERTIES)
|
||||
hiveSyncConfig
|
||||
}
|
||||
|
||||
private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path,
|
||||
schema: StructType): Boolean = {
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
|
||||
var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
|
||||
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
|
||||
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
|
||||
// for backward compatibility
|
||||
if (hiveSyncEnabled) {
|
||||
metaSyncEnabled = true
|
||||
syncClientToolClassSet += classOf[HiveSyncTool].getName
|
||||
}
|
||||
var metaSyncSuccess = true
|
||||
if (metaSyncEnabled) {
|
||||
val fs = basePath.getFileSystem(spark.sessionState.newHadoopConf())
|
||||
syncClientToolClassSet.foreach(impl => {
|
||||
val syncSuccess = impl.trim match {
|
||||
case "org.apache.hudi.hive.HiveSyncTool" => {
|
||||
log.info("Syncing to Hive Metastore (URL: " + hoodieConfig.getString(HIVE_URL) + ")")
|
||||
syncHive(basePath, fs, hoodieConfig, spark.sessionState.conf)
|
||||
true
|
||||
}
|
||||
case _ => {
|
||||
val properties = new Properties()
|
||||
properties.putAll(hoodieConfig.getProps)
|
||||
properties.put("basePath", basePath.toString)
|
||||
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
|
||||
syncHoodie.syncHoodieTable()
|
||||
true
|
||||
}
|
||||
}
|
||||
metaSyncSuccess = metaSyncSuccess && syncSuccess
|
||||
})
|
||||
}
|
||||
metaSyncSuccess
|
||||
}
|
||||
|
||||
/**
|
||||
* Group all table/action specific information into a case class.
|
||||
*/
|
||||
case class TableInstantInfo(basePath: Path, instantTime: String, commitActionType: String, operation: WriteOperationType)
|
||||
|
||||
private def commitAndPerformPostOperations(spark: SparkSession,
|
||||
schema: StructType,
|
||||
writeResult: HoodieWriteResult,
|
||||
parameters: Map[String, String],
|
||||
client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
|
||||
tableConfig: HoodieTableConfig,
|
||||
jsc: JavaSparkContext,
|
||||
tableInstantInfo: TableInstantInfo
|
||||
): (Boolean, common.util.Option[java.lang.String], common.util.Option[java.lang.String]) = {
|
||||
if(writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors).isEmpty()) {
|
||||
log.info("Proceeding to commit the write.")
|
||||
val metaMap = parameters.filter(kv =>
|
||||
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX.key)))
|
||||
val commitSuccess =
|
||||
client.commit(tableInstantInfo.instantTime, writeResult.getWriteStatuses,
|
||||
common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))),
|
||||
tableInstantInfo.commitActionType,
|
||||
writeResult.getPartitionToReplaceFileIds)
|
||||
|
||||
if (commitSuccess) {
|
||||
log.info("Commit " + tableInstantInfo.instantTime + " successful!")
|
||||
}
|
||||
else {
|
||||
log.info("Commit " + tableInstantInfo.instantTime + " failed!")
|
||||
}
|
||||
|
||||
val asyncCompactionEnabled = isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())
|
||||
val compactionInstant: common.util.Option[java.lang.String] =
|
||||
if (asyncCompactionEnabled) {
|
||||
client.scheduleCompaction(common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
} else {
|
||||
common.util.Option.empty()
|
||||
}
|
||||
|
||||
log.info(s"Compaction Scheduled is $compactionInstant")
|
||||
|
||||
val asyncClusteringEnabled = isAsyncClusteringEnabled(client, parameters)
|
||||
val clusteringInstant: common.util.Option[java.lang.String] =
|
||||
if (asyncClusteringEnabled) {
|
||||
client.scheduleClustering(common.util.Option.of(new java.util.HashMap[String, String](mapAsJavaMap(metaMap))))
|
||||
} else {
|
||||
common.util.Option.empty()
|
||||
}
|
||||
|
||||
log.info(s"Clustering Scheduled is $clusteringInstant")
|
||||
|
||||
val metaSyncSuccess = metaSync(spark, HoodieWriterUtils.convertMapToHoodieConfig(parameters),
|
||||
tableInstantInfo.basePath, schema)
|
||||
|
||||
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
|
||||
if (!asyncCompactionEnabled && !asyncClusteringEnabled) {
|
||||
client.close()
|
||||
}
|
||||
(commitSuccess && metaSyncSuccess, compactionInstant, clusteringInstant)
|
||||
} else {
|
||||
log.error(s"${tableInstantInfo.operation} failed with errors")
|
||||
if (log.isTraceEnabled) {
|
||||
log.trace("Printing out the top 100 errors")
|
||||
writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors)
|
||||
.take(100)
|
||||
.foreach(ws => {
|
||||
log.trace("Global error :", ws.getGlobalError)
|
||||
if (ws.getErrors.size() > 0) {
|
||||
ws.getErrors.foreach(kt =>
|
||||
log.trace(s"Error for key: ${kt._1}", kt._2))
|
||||
}
|
||||
})
|
||||
}
|
||||
(false, common.util.Option.empty(), common.util.Option.empty())
|
||||
}
|
||||
}
|
||||
|
||||
private def isAsyncCompactionEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
|
||||
tableConfig: HoodieTableConfig,
|
||||
parameters: Map[String, String], configuration: Configuration) : Boolean = {
|
||||
log.info(s"Config.inlineCompactionEnabled ? ${client.getConfig.inlineCompactionEnabled}")
|
||||
if (asyncCompactionTriggerFnDefined && !client.getConfig.inlineCompactionEnabled
|
||||
&& parameters.get(ASYNC_COMPACT_ENABLE.key).exists(r => r.toBoolean)) {
|
||||
tableConfig.getTableType == HoodieTableType.MERGE_ON_READ
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
private def isAsyncClusteringEnabled(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]],
|
||||
parameters: Map[String, String]) : Boolean = {
|
||||
log.info(s"Config.asyncClusteringEnabled ? ${client.getConfig.isAsyncClusteringEnabled}")
|
||||
asyncClusteringTriggerFnDefined && client.getConfig.isAsyncClusteringEnabled &&
|
||||
parameters.get(ASYNC_CLUSTERING_ENABLE.key).exists(r => r.toBoolean)
|
||||
}
|
||||
|
||||
private def getHoodieTableConfig(sparkContext: SparkContext,
|
||||
tablePath: String,
|
||||
hoodieTableConfigOpt: Option[HoodieTableConfig]): HoodieTableConfig = {
|
||||
if (tableExists) {
|
||||
hoodieTableConfigOpt.getOrElse(
|
||||
HoodieTableMetaClient.builder().setConf(sparkContext.hadoopConfiguration).setBasePath(tablePath)
|
||||
.build().getTableConfig)
|
||||
} else {
|
||||
null
|
||||
}
|
||||
}
|
||||
|
||||
private def mergeParamsAndGetHoodieConfig(optParams: Map[String, String],
|
||||
tableConfig: HoodieTableConfig): (Map[String, String], HoodieConfig) = {
|
||||
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(optParams)
|
||||
val mergedParams = mutable.Map.empty ++ HoodieWriterUtils.parametersWithWriteDefaults(translatedOptions)
|
||||
if (!mergedParams.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
|
||||
&& mergedParams.contains(KEYGENERATOR_CLASS_NAME.key)) {
|
||||
mergedParams(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = mergedParams(KEYGENERATOR_CLASS_NAME.key)
|
||||
}
|
||||
if (null != tableConfig) {
|
||||
tableConfig.getProps.foreach { case (key, value) =>
|
||||
mergedParams(key) = value
|
||||
}
|
||||
}
|
||||
val params = mergedParams.toMap
|
||||
(params, HoodieWriterUtils.convertMapToHoodieConfig(params))
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,247 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.hudi.async.{AsyncClusteringService, AsyncCompactService, SparkStreamingAsyncClusteringService, SparkStreamingAsyncCompactService}
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload
|
||||
import org.apache.hudi.common.table.marker.MarkerType
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.util.{ClusteringUtils, CompactionUtils}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieCorruptedDataException
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.execution.streaming.Sink
|
||||
import org.apache.spark.sql.streaming.OutputMode
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
||||
|
||||
import java.lang
|
||||
import java.util.function.Function
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.util.{Failure, Success, Try}
|
||||
|
||||
class HoodieStreamingSink(sqlContext: SQLContext,
|
||||
options: Map[String, String],
|
||||
partitionColumns: Seq[String],
|
||||
outputMode: OutputMode)
|
||||
extends Sink
|
||||
with Serializable {
|
||||
@volatile private var latestBatchId = -1L
|
||||
|
||||
private val log = LogManager.getLogger(classOf[HoodieStreamingSink])
|
||||
|
||||
private val retryCnt = options.getOrDefault(DataSourceWriteOptions.STREAMING_RETRY_CNT.key,
|
||||
DataSourceWriteOptions.STREAMING_RETRY_CNT.defaultValue).toInt
|
||||
private val retryIntervalMs = options.getOrDefault(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS.key,
|
||||
DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS.defaultValue).toLong
|
||||
private val ignoreFailedBatch = options.getOrDefault(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.key,
|
||||
DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.defaultValue).toBoolean
|
||||
|
||||
private var isAsyncCompactorServiceShutdownAbnormally = false
|
||||
private var isAsyncClusteringServiceShutdownAbnormally = false
|
||||
|
||||
private val mode =
|
||||
if (outputMode == OutputMode.Append()) {
|
||||
SaveMode.Append
|
||||
} else {
|
||||
SaveMode.Overwrite
|
||||
}
|
||||
|
||||
private var asyncCompactorService : AsyncCompactService = _
|
||||
private var asyncClusteringService: AsyncClusteringService = _
|
||||
private var writeClient : Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty
|
||||
private var hoodieTableConfig : Option[HoodieTableConfig] = Option.empty
|
||||
|
||||
override def addBatch(batchId: Long, data: DataFrame): Unit = this.synchronized {
|
||||
if (isAsyncCompactorServiceShutdownAbnormally) {
|
||||
throw new IllegalStateException("Async Compactor shutdown unexpectedly")
|
||||
}
|
||||
if (isAsyncClusteringServiceShutdownAbnormally) {
|
||||
log.error("Async clustering service shutdown unexpectedly")
|
||||
throw new IllegalStateException("Async clustering service shutdown unexpectedly")
|
||||
}
|
||||
// Override to use direct markers. In Structured streaming, timeline server is closed after
|
||||
// first micro-batch and subsequent micro-batches do not have timeline server running.
|
||||
// Thus, we can't use timeline-server-based markers.
|
||||
val updatedOptions = options.updated(HoodieWriteConfig.MARKERS_TYPE.key(), MarkerType.DIRECT.name())
|
||||
|
||||
retry(retryCnt, retryIntervalMs)(
|
||||
Try(
|
||||
HoodieSparkSqlWriter.write(
|
||||
sqlContext, mode, updatedOptions, data, hoodieTableConfig, writeClient, Some(triggerAsyncCompactor), Some(triggerAsyncClustering))
|
||||
) match {
|
||||
case Success((true, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) =>
|
||||
log.info(s"Micro batch id=$batchId succeeded"
|
||||
+ (commitOps.isPresent match {
|
||||
case true => s" for commit=${commitOps.get()}"
|
||||
case _ => s" with no new commits"
|
||||
}))
|
||||
writeClient = Some(client)
|
||||
hoodieTableConfig = Some(tableConfig)
|
||||
if (compactionInstantOps.isPresent) {
|
||||
asyncCompactorService.enqueuePendingAsyncServiceInstant(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionInstantOps.get()))
|
||||
}
|
||||
if (clusteringInstant.isPresent) {
|
||||
asyncClusteringService.enqueuePendingAsyncServiceInstant(new HoodieInstant(
|
||||
State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringInstant.get()
|
||||
))
|
||||
}
|
||||
Success((true, commitOps, compactionInstantOps))
|
||||
case Failure(e) =>
|
||||
// clean up persist rdds in the write process
|
||||
data.sparkSession.sparkContext.getPersistentRDDs
|
||||
.foreach {
|
||||
case (id, rdd) =>
|
||||
try {
|
||||
rdd.unpersist()
|
||||
} catch {
|
||||
case t: Exception => log.warn("Got excepting trying to unpersist rdd", t)
|
||||
}
|
||||
}
|
||||
log.error(s"Micro batch id=$batchId threw following exception: ", e)
|
||||
if (ignoreFailedBatch) {
|
||||
log.info(s"Ignore the exception and move on streaming as per " +
|
||||
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.key} configuration")
|
||||
Success((true, None, None))
|
||||
} else {
|
||||
if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...")
|
||||
Failure(e)
|
||||
}
|
||||
case Success((false, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) =>
|
||||
log.error(s"Micro batch id=$batchId ended up with errors"
|
||||
+ (commitOps.isPresent match {
|
||||
case true => s" for commit=${commitOps.get()}"
|
||||
case _ => s""
|
||||
}))
|
||||
if (ignoreFailedBatch) {
|
||||
log.info(s"Ignore the errors and move on streaming as per " +
|
||||
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.key} configuration")
|
||||
Success((true, None, None))
|
||||
} else {
|
||||
if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...")
|
||||
Failure(new HoodieCorruptedDataException(s"Micro batch id=$batchId ended up with errors"))
|
||||
}
|
||||
}
|
||||
) match {
|
||||
case Failure(e) =>
|
||||
if (!ignoreFailedBatch) {
|
||||
log.error(s"Micro batch id=$batchId threw following expections," +
|
||||
s"aborting streaming app to avoid data loss: ", e)
|
||||
// spark sometimes hangs upon exceptions and keep on hold of the executors
|
||||
// this is to force exit upon errors / exceptions and release all executors
|
||||
// will require redeployment / supervise mode to restart the streaming
|
||||
reset(true)
|
||||
System.exit(1)
|
||||
}
|
||||
case Success(_) =>
|
||||
log.info(s"Micro batch id=$batchId succeeded")
|
||||
}
|
||||
}
|
||||
|
||||
override def toString: String = s"HoodieStreamingSink[${options("path")}]"
|
||||
|
||||
@annotation.tailrec
|
||||
private def retry[T](n: Int, waitInMillis: Long)(fn: => Try[T]): Try[T] = {
|
||||
fn match {
|
||||
case x: Success[T] =>
|
||||
x
|
||||
case _ if n > 1 =>
|
||||
Thread.sleep(waitInMillis)
|
||||
retry(n - 1, waitInMillis * 2)(fn)
|
||||
case f =>
|
||||
reset(false)
|
||||
f
|
||||
}
|
||||
}
|
||||
|
||||
protected def triggerAsyncCompactor(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
|
||||
if (null == asyncCompactorService) {
|
||||
log.info("Triggering Async compaction !!")
|
||||
asyncCompactorService = new SparkStreamingAsyncCompactService(new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
|
||||
client)
|
||||
asyncCompactorService.start(new Function[java.lang.Boolean, java.lang.Boolean] {
|
||||
override def apply(errored: lang.Boolean): lang.Boolean = {
|
||||
log.info(s"Async Compactor shutdown. Errored ? $errored")
|
||||
isAsyncCompactorServiceShutdownAbnormally = errored
|
||||
reset(false)
|
||||
log.info("Done resetting write client.")
|
||||
true
|
||||
}
|
||||
})
|
||||
|
||||
// Add Shutdown Hook
|
||||
Runtime.getRuntime.addShutdownHook(new Thread(new Runnable {
|
||||
override def run(): Unit = reset(true)
|
||||
}))
|
||||
|
||||
// First time, scan .hoodie folder and get all pending compactions
|
||||
val metaClient = HoodieTableMetaClient.builder().setConf(sqlContext.sparkContext.hadoopConfiguration)
|
||||
.setBasePath(client.getConfig.getBasePath).build()
|
||||
val pendingInstants :java.util.List[HoodieInstant] =
|
||||
CompactionUtils.getPendingCompactionInstantTimes(metaClient)
|
||||
pendingInstants.foreach((h : HoodieInstant) => asyncCompactorService.enqueuePendingAsyncServiceInstant(h))
|
||||
}
|
||||
}
|
||||
|
||||
protected def triggerAsyncClustering(client: SparkRDDWriteClient[HoodieRecordPayload[Nothing]]): Unit = {
|
||||
if (null == asyncClusteringService) {
|
||||
log.info("Triggering async clustering!")
|
||||
asyncClusteringService = new SparkStreamingAsyncClusteringService(client)
|
||||
asyncClusteringService.start(new Function[java.lang.Boolean, java.lang.Boolean] {
|
||||
override def apply(errored: lang.Boolean): lang.Boolean = {
|
||||
log.info(s"Async clustering service shutdown. Errored ? $errored")
|
||||
isAsyncClusteringServiceShutdownAbnormally = errored
|
||||
reset(false)
|
||||
true
|
||||
}
|
||||
})
|
||||
|
||||
// Add Shutdown Hook
|
||||
Runtime.getRuntime.addShutdownHook(new Thread(new Runnable {
|
||||
override def run(): Unit = reset(true)
|
||||
}))
|
||||
|
||||
// First time, scan .hoodie folder and get all pending clustering instants
|
||||
val metaClient = HoodieTableMetaClient.builder().setConf(sqlContext.sparkContext.hadoopConfiguration)
|
||||
.setBasePath(client.getConfig.getBasePath).build()
|
||||
val pendingInstants :java.util.List[HoodieInstant] = ClusteringUtils.getPendingClusteringInstantTimes(metaClient)
|
||||
pendingInstants.foreach((h : HoodieInstant) => asyncClusteringService.enqueuePendingAsyncServiceInstant(h))
|
||||
}
|
||||
}
|
||||
|
||||
private def reset(force: Boolean) : Unit = this.synchronized {
|
||||
if (asyncCompactorService != null) {
|
||||
asyncCompactorService.shutdown(force)
|
||||
asyncCompactorService = null
|
||||
}
|
||||
|
||||
if (asyncClusteringService != null) {
|
||||
asyncClusteringService.shutdown(force)
|
||||
asyncClusteringService = null
|
||||
}
|
||||
|
||||
if (writeClient.isDefined) {
|
||||
writeClient.get.close()
|
||||
writeClient = Option.empty
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,196 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.hudi.DataSourceOptionsHelper.allAlternatives
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
|
||||
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.hudi.command.SqlKeyGenerator
|
||||
|
||||
import scala.collection.JavaConversions.mapAsJavaMap
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
/**
|
||||
* WriterUtils to assist in write path in Datasource and tests.
|
||||
*/
|
||||
object HoodieWriterUtils {
|
||||
|
||||
def javaParametersWithWriteDefaults(parameters: java.util.Map[String, String]): java.util.Map[String, String] = {
|
||||
mapAsJavaMap(parametersWithWriteDefaults(parameters.asScala.toMap))
|
||||
}
|
||||
|
||||
/**
|
||||
* Add default options for unspecified write options keys.
|
||||
*
|
||||
* @param parameters
|
||||
* @return
|
||||
*/
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala
|
||||
val props = new Properties()
|
||||
props.putAll(parameters)
|
||||
val hoodieConfig: HoodieConfig = new HoodieConfig(props)
|
||||
hoodieConfig.setDefaultValue(OPERATION)
|
||||
hoodieConfig.setDefaultValue(TABLE_TYPE)
|
||||
hoodieConfig.setDefaultValue(PRECOMBINE_FIELD)
|
||||
hoodieConfig.setDefaultValue(PAYLOAD_CLASS_NAME)
|
||||
hoodieConfig.setDefaultValue(RECORDKEY_FIELD)
|
||||
hoodieConfig.setDefaultValue(KEYGENERATOR_CLASS_NAME)
|
||||
hoodieConfig.setDefaultValue(ENABLE)
|
||||
hoodieConfig.setDefaultValue(COMMIT_METADATA_KEYPREFIX)
|
||||
hoodieConfig.setDefaultValue(INSERT_DROP_DUPS)
|
||||
hoodieConfig.setDefaultValue(STREAMING_RETRY_CNT)
|
||||
hoodieConfig.setDefaultValue(STREAMING_RETRY_INTERVAL_MS)
|
||||
hoodieConfig.setDefaultValue(STREAMING_IGNORE_FAILED_BATCH)
|
||||
hoodieConfig.setDefaultValue(META_SYNC_CLIENT_TOOL_CLASS_NAME)
|
||||
hoodieConfig.setDefaultValue(HIVE_SYNC_ENABLED)
|
||||
hoodieConfig.setDefaultValue(META_SYNC_ENABLED)
|
||||
hoodieConfig.setDefaultValue(HIVE_DATABASE)
|
||||
hoodieConfig.setDefaultValue(HIVE_TABLE)
|
||||
hoodieConfig.setDefaultValue(HIVE_BASE_FILE_FORMAT)
|
||||
hoodieConfig.setDefaultValue(HIVE_USER)
|
||||
hoodieConfig.setDefaultValue(HIVE_PASS)
|
||||
hoodieConfig.setDefaultValue(HIVE_URL)
|
||||
hoodieConfig.setDefaultValue(HIVE_PARTITION_FIELDS)
|
||||
hoodieConfig.setDefaultValue(HIVE_PARTITION_EXTRACTOR_CLASS)
|
||||
hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING)
|
||||
hoodieConfig.setDefaultValue(HIVE_USE_JDBC)
|
||||
hoodieConfig.setDefaultValue(HIVE_CREATE_MANAGED_TABLE)
|
||||
hoodieConfig.setDefaultValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE)
|
||||
hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE)
|
||||
hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE)
|
||||
hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE)
|
||||
hoodieConfig.setDefaultValue(ENABLE_ROW_WRITER)
|
||||
hoodieConfig.setDefaultValue(RECONCILE_SCHEMA)
|
||||
hoodieConfig.setDefaultValue(DROP_PARTITION_COLUMNS)
|
||||
hoodieConfig.setDefaultValue(KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED)
|
||||
Map() ++ hoodieConfig.getProps.asScala ++ globalProps ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
params.foreach(kv => props.setProperty(kv._1, kv._2))
|
||||
props
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the partition columns to stored to hoodie.properties.
|
||||
* @param parameters
|
||||
* @return
|
||||
*/
|
||||
def getPartitionColumns(parameters: Map[String, String]): String = {
|
||||
val props = new Properties()
|
||||
props.putAll(parameters.asJava)
|
||||
HoodieSparkUtils.getPartitionColumns(props)
|
||||
}
|
||||
|
||||
def convertMapToHoodieConfig(parameters: Map[String, String]): HoodieConfig = {
|
||||
val properties = new Properties()
|
||||
properties.putAll(mapAsJavaMap(parameters))
|
||||
new HoodieConfig(properties)
|
||||
}
|
||||
|
||||
def getOriginKeyGenerator(parameters: Map[String, String]): String = {
|
||||
val kg = parameters.getOrElse(KEYGENERATOR_CLASS_NAME.key(), null)
|
||||
if (classOf[SqlKeyGenerator].getCanonicalName == kg) {
|
||||
parameters.getOrElse(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null)
|
||||
} else {
|
||||
kg
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Detects conflicts between new parameters and existing table configurations
|
||||
*/
|
||||
def validateTableConfig(spark: SparkSession, params: Map[String, String],
|
||||
tableConfig: HoodieConfig): Unit = {
|
||||
val resolver = spark.sessionState.conf.resolver
|
||||
val diffConfigs = StringBuilder.newBuilder
|
||||
params.foreach { case (key, value) =>
|
||||
val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key)
|
||||
if (null != existingValue && !resolver(existingValue, value)) {
|
||||
diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n")
|
||||
}
|
||||
}
|
||||
|
||||
if (null != tableConfig) {
|
||||
val datasourceRecordKey = params.getOrElse(RECORDKEY_FIELD.key(), null)
|
||||
val tableConfigRecordKey = tableConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS)
|
||||
if (null != datasourceRecordKey && null != tableConfigRecordKey
|
||||
&& datasourceRecordKey != tableConfigRecordKey) {
|
||||
diffConfigs.append(s"RecordKey:\t$datasourceRecordKey\t$tableConfigRecordKey\n")
|
||||
}
|
||||
|
||||
val datasourcePreCombineKey = params.getOrElse(PRECOMBINE_FIELD.key(), null)
|
||||
val tableConfigPreCombineKey = tableConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD)
|
||||
if (null != datasourcePreCombineKey && null != tableConfigPreCombineKey
|
||||
&& datasourcePreCombineKey != tableConfigPreCombineKey) {
|
||||
diffConfigs.append(s"PreCombineKey:\t$datasourcePreCombineKey\t$tableConfigPreCombineKey\n")
|
||||
}
|
||||
|
||||
val datasourceKeyGen = getOriginKeyGenerator(params)
|
||||
val tableConfigKeyGen = tableConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)
|
||||
if (null != datasourceKeyGen && null != tableConfigKeyGen
|
||||
&& datasourceKeyGen != tableConfigKeyGen) {
|
||||
diffConfigs.append(s"KeyGenerator:\t$datasourceKeyGen\t$tableConfigKeyGen\n")
|
||||
}
|
||||
}
|
||||
|
||||
if (diffConfigs.nonEmpty) {
|
||||
diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n")
|
||||
throw new HoodieException(diffConfigs.toString.trim)
|
||||
}
|
||||
}
|
||||
|
||||
private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieConfig, key: String): String = {
|
||||
if (null == tableConfig) {
|
||||
null
|
||||
} else {
|
||||
if (allAlternatives.contains(key)) {
|
||||
tableConfig.getString(allAlternatives(key))
|
||||
} else {
|
||||
tableConfig.getString(key)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
val sparkDatasourceConfigsToTableConfigsMap = Map(
|
||||
TABLE_NAME -> HoodieTableConfig.NAME,
|
||||
TABLE_TYPE -> HoodieTableConfig.TYPE,
|
||||
PRECOMBINE_FIELD -> HoodieTableConfig.PRECOMBINE_FIELD,
|
||||
PARTITIONPATH_FIELD -> HoodieTableConfig.PARTITION_FIELDS,
|
||||
RECORDKEY_FIELD -> HoodieTableConfig.RECORDKEY_FIELDS,
|
||||
PAYLOAD_CLASS_NAME -> HoodieTableConfig.PAYLOAD_CLASS_NAME
|
||||
)
|
||||
def mappingSparkDatasourceConfigsToTableConfigs(options: Map[String, String]): Map[String, String] = {
|
||||
val includingTableConfigs = scala.collection.mutable.Map() ++ options
|
||||
sparkDatasourceConfigsToTableConfigsMap.foreach(kv => {
|
||||
if (options.containsKey(kv._1.key)) {
|
||||
includingTableConfigs(kv._2.key) = options(kv._1.key)
|
||||
includingTableConfigs.remove(kv._1.key)
|
||||
}
|
||||
})
|
||||
includingTableConfigs.toMap
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,194 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.avro.Schema
|
||||
|
||||
import java.util.stream.Collectors
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieRecord, HoodieReplaceCommitMetadata, HoodieTableType}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hadoop.fs.GlobPattern
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.sources.{BaseRelation, TableScan}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, Row, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
* Relation, that implements the Hoodie incremental view.
|
||||
*
|
||||
* Implemented for Copy_on_write storage.
|
||||
*
|
||||
*/
|
||||
class IncrementalRelation(val sqlContext: SQLContext,
|
||||
val optParams: Map[String, String],
|
||||
val userSchema: StructType,
|
||||
val metaClient: HoodieTableMetaClient) extends BaseRelation with TableScan {
|
||||
|
||||
private val log = LogManager.getLogger(classOf[IncrementalRelation])
|
||||
|
||||
val skeletonSchema: StructType = HoodieSparkUtils.getMetaSchema
|
||||
private val basePath = metaClient.getBasePath
|
||||
// TODO : Figure out a valid HoodieWriteConfig
|
||||
private val hoodieTable = HoodieSparkTable.create(HoodieWriteConfig.newBuilder().withPath(basePath).build(),
|
||||
new HoodieSparkEngineContext(new JavaSparkContext(sqlContext.sparkContext)),
|
||||
metaClient)
|
||||
private val commitTimeline = hoodieTable.getMetaClient.getCommitTimeline.filterCompletedInstants()
|
||||
if (commitTimeline.empty()) {
|
||||
throw new HoodieException("No instants to incrementally pull")
|
||||
}
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME.key)) {
|
||||
throw new HoodieException(s"Specify the begin instant time to pull from using " +
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME.key}")
|
||||
}
|
||||
if (!metaClient.getTableConfig.populateMetaFields()) {
|
||||
throw new HoodieException("Incremental queries are not supported when meta fields are disabled")
|
||||
}
|
||||
|
||||
val useEndInstantSchema = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME.key,
|
||||
DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME.defaultValue).toBoolean
|
||||
|
||||
private val lastInstant = commitTimeline.lastInstant().get()
|
||||
|
||||
private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(), lastInstant.getTimestamp))
|
||||
private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
|
||||
|
||||
// use schema from a file produced in the end/latest instant
|
||||
val usedSchema: StructType = {
|
||||
log.info("Inferring schema..")
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val tableSchema = if (useEndInstantSchema) {
|
||||
if (commitsToReturn.isEmpty) schemaResolver.getTableAvroSchemaWithoutMetadataFields() else
|
||||
schemaResolver.getTableAvroSchemaWithoutMetadataFields(commitsToReturn.last)
|
||||
} else {
|
||||
schemaResolver.getTableAvroSchemaWithoutMetadataFields()
|
||||
}
|
||||
if (tableSchema.getType == Schema.Type.NULL) {
|
||||
// if there is only one commit in the table and is an empty commit without schema, return empty RDD here
|
||||
StructType(Nil)
|
||||
} else {
|
||||
val dataSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableSchema)
|
||||
StructType(skeletonSchema.fields ++ dataSchema.fields)
|
||||
}
|
||||
}
|
||||
|
||||
private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS.key,
|
||||
DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS.defaultValue).split(",").filter(!_.isEmpty)
|
||||
|
||||
override def schema: StructType = usedSchema
|
||||
|
||||
override def buildScan(): RDD[Row] = {
|
||||
if (usedSchema == StructType(Nil)) {
|
||||
// if first commit in a table is an empty commit without schema, return empty RDD here
|
||||
sqlContext.sparkContext.emptyRDD[Row]
|
||||
} else {
|
||||
val regularFileIdToFullPath = mutable.HashMap[String, String]()
|
||||
var metaBootstrapFileIdToFullPath = mutable.HashMap[String, String]()
|
||||
|
||||
// create Replaced file group
|
||||
val replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline
|
||||
val replacedFile = replacedTimeline.getInstants.collect(Collectors.toList[HoodieInstant]).flatMap { instant =>
|
||||
val replaceMetadata = HoodieReplaceCommitMetadata.
|
||||
fromBytes(metaClient.getActiveTimeline.getInstantDetails(instant).get, classOf[HoodieReplaceCommitMetadata])
|
||||
replaceMetadata.getPartitionToReplaceFileIds.entrySet().flatMap { entry =>
|
||||
entry.getValue.map { e =>
|
||||
val fullPath = FSUtils.getPartitionPath(basePath, entry.getKey).toString
|
||||
(e, fullPath)
|
||||
}
|
||||
}
|
||||
}.toMap
|
||||
|
||||
for (commit <- commitsToReturn) {
|
||||
val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit)
|
||||
.get, classOf[HoodieCommitMetadata])
|
||||
|
||||
if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) {
|
||||
metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap.filterNot { case (k, v) =>
|
||||
replacedFile.contains(k) && v.startsWith(replacedFile(k))
|
||||
}
|
||||
} else {
|
||||
regularFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).toMap.filterNot { case (k, v) =>
|
||||
replacedFile.contains(k) && v.startsWith(replacedFile(k))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (metaBootstrapFileIdToFullPath.nonEmpty) {
|
||||
// filer out meta bootstrap files that have had more commits since metadata bootstrap
|
||||
metaBootstrapFileIdToFullPath = metaBootstrapFileIdToFullPath
|
||||
.filterNot(fileIdFullPath => regularFileIdToFullPath.contains(fileIdFullPath._1))
|
||||
}
|
||||
|
||||
val pathGlobPattern = optParams.getOrElse(
|
||||
DataSourceReadOptions.INCR_PATH_GLOB.key,
|
||||
DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)
|
||||
val (filteredRegularFullPaths, filteredMetaBootstrapFullPaths) = {
|
||||
if (!pathGlobPattern.equals(DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)) {
|
||||
val globMatcher = new GlobPattern("*" + pathGlobPattern)
|
||||
(regularFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values,
|
||||
metaBootstrapFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values)
|
||||
} else {
|
||||
(regularFileIdToFullPath.values, metaBootstrapFileIdToFullPath.values)
|
||||
}
|
||||
}
|
||||
// unset the path filter, otherwise if end_instant_time is not the latest instant, path filter set for RO view
|
||||
// will filter out all the files incorrectly.
|
||||
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class")
|
||||
val sOpts = optParams.filter(p => !p._1.equalsIgnoreCase("path"))
|
||||
if (filteredRegularFullPaths.isEmpty && filteredMetaBootstrapFullPaths.isEmpty) {
|
||||
sqlContext.sparkContext.emptyRDD[Row]
|
||||
} else {
|
||||
log.info("Additional Filters to be applied to incremental source are :" + filters)
|
||||
|
||||
var df: DataFrame = sqlContext.createDataFrame(sqlContext.sparkContext.emptyRDD[Row], usedSchema)
|
||||
|
||||
if (metaBootstrapFileIdToFullPath.nonEmpty) {
|
||||
df = sqlContext.sparkSession.read
|
||||
.format("hudi")
|
||||
.schema(usedSchema)
|
||||
.option(DataSourceReadOptions.READ_PATHS.key, filteredMetaBootstrapFullPaths.mkString(","))
|
||||
.load()
|
||||
}
|
||||
|
||||
if (regularFileIdToFullPath.nonEmpty) {
|
||||
df = df.union(sqlContext.read.options(sOpts)
|
||||
.schema(usedSchema)
|
||||
.parquet(filteredRegularFullPaths.toList: _*)
|
||||
.filter(String.format("%s >= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.head.getTimestamp))
|
||||
.filter(String.format("%s <= '%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD,
|
||||
commitsToReturn.last.getTimestamp)))
|
||||
}
|
||||
|
||||
filters.foldLeft(df)((e, f) => e.filter(f)).rdd
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,226 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.listAffectedFilesForCommits
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getCommitMetadata
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils.getWritePartitionPaths
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
|
||||
import org.apache.hadoop.fs.{GlobPattern, Path}
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{Row, SQLContext}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
* Experimental.
|
||||
* Relation, that implements the Hoodie incremental view for Merge On Read table.
|
||||
*
|
||||
*/
|
||||
class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
|
||||
val optParams: Map[String, String],
|
||||
val userSchema: StructType,
|
||||
val metaClient: HoodieTableMetaClient)
|
||||
extends BaseRelation with PrunedFilteredScan {
|
||||
|
||||
private val log = LogManager.getLogger(classOf[MergeOnReadIncrementalRelation])
|
||||
private val conf = sqlContext.sparkContext.hadoopConfiguration
|
||||
private val jobConf = new JobConf(conf)
|
||||
private val commitTimeline = metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants()
|
||||
if (commitTimeline.empty()) {
|
||||
throw new HoodieException("No instants to incrementally pull")
|
||||
}
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME.key)) {
|
||||
throw new HoodieException(s"Specify the begin instant time to pull from using " +
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME.key}")
|
||||
}
|
||||
if (!metaClient.getTableConfig.populateMetaFields()) {
|
||||
throw new HoodieException("Incremental queries are not supported when meta fields are disabled")
|
||||
}
|
||||
|
||||
private val lastInstant = commitTimeline.lastInstant().get()
|
||||
private val mergeType = optParams.getOrElse(
|
||||
DataSourceReadOptions.REALTIME_MERGE.key,
|
||||
DataSourceReadOptions.REALTIME_MERGE.defaultValue)
|
||||
|
||||
private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key, lastInstant.getTimestamp))
|
||||
log.debug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => f.toString).mkString(",")}")
|
||||
private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
|
||||
private val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
private val tableAvroSchema = schemaUtil.getTableAvroSchema
|
||||
private val tableStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
|
||||
private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf)
|
||||
private val fileIndex = if (commitsToReturn.isEmpty) List() else buildFileIndex()
|
||||
private val preCombineField = {
|
||||
val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField
|
||||
if (preCombineFieldFromTableConfig != null) {
|
||||
Some(preCombineFieldFromTableConfig)
|
||||
} else {
|
||||
// get preCombineFiled from the options if this is a old table which have not store
|
||||
// the field to hoodie.properties
|
||||
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
|
||||
}
|
||||
}
|
||||
override def schema: StructType = tableStructSchema
|
||||
|
||||
override def needConversion: Boolean = false
|
||||
|
||||
override def unhandledFilters(filters: Array[Filter]): Array[Filter] = {
|
||||
if (fileIndex.isEmpty) {
|
||||
filters
|
||||
} else {
|
||||
val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
|
||||
val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)
|
||||
val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
|
||||
filters :+ isNotNullFilter :+ largerThanFilter :+ lessThanFilter
|
||||
}
|
||||
}
|
||||
|
||||
override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
|
||||
if (fileIndex.isEmpty) {
|
||||
sqlContext.sparkContext.emptyRDD[Row]
|
||||
} else {
|
||||
log.debug(s"buildScan requiredColumns = ${requiredColumns.mkString(",")}")
|
||||
log.debug(s"buildScan filters = ${filters.mkString(",")}")
|
||||
// config to ensure the push down filter for parquet will be applied.
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.filterPushdown", "true")
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.recordLevelFilter.enabled", "true")
|
||||
sqlContext.sparkSession.sessionState.conf.setConfString("spark.sql.parquet.enableVectorizedReader", "false")
|
||||
val pushDownFilter = {
|
||||
val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD)
|
||||
val largerThanFilter = GreaterThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.head.getTimestamp)
|
||||
val lessThanFilter = LessThanOrEqual(HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitsToReturn.last.getTimestamp)
|
||||
filters :+ isNotNullFilter :+ largerThanFilter :+ lessThanFilter
|
||||
}
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
MergeOnReadSnapshotRelation.getRequiredSchema(tableAvroSchema, requiredColumns)
|
||||
|
||||
val hoodieTableState = HoodieMergeOnReadTableState(
|
||||
tableStructSchema,
|
||||
requiredStructSchema,
|
||||
tableAvroSchema.toString,
|
||||
requiredAvroSchema.toString,
|
||||
fileIndex,
|
||||
preCombineField,
|
||||
Option.empty
|
||||
)
|
||||
val fullSchemaParquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
dataSchema = tableStructSchema,
|
||||
partitionSchema = StructType(Nil),
|
||||
requiredSchema = tableStructSchema,
|
||||
filters = pushDownFilter,
|
||||
options = optParams,
|
||||
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
)
|
||||
val requiredSchemaParquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
dataSchema = tableStructSchema,
|
||||
partitionSchema = StructType(Nil),
|
||||
requiredSchema = requiredStructSchema,
|
||||
filters = pushDownFilter,
|
||||
options = optParams,
|
||||
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
)
|
||||
|
||||
val rdd = new HoodieMergeOnReadRDD(
|
||||
sqlContext.sparkContext,
|
||||
jobConf,
|
||||
fullSchemaParquetReader,
|
||||
requiredSchemaParquetReader,
|
||||
hoodieTableState
|
||||
)
|
||||
rdd.asInstanceOf[RDD[Row]]
|
||||
}
|
||||
}
|
||||
|
||||
def buildFileIndex(): List[HoodieMergeOnReadFileSplit] = {
|
||||
val metadataList = commitsToReturn.map(instant => getCommitMetadata(instant, commitsTimelineToReturn))
|
||||
val affectedFileStatus = listAffectedFilesForCommits(new Path(metaClient.getBasePath), metadataList)
|
||||
val fsView = new HoodieTableFileSystemView(metaClient, commitsTimelineToReturn, affectedFileStatus)
|
||||
|
||||
// Iterate partitions to create splits
|
||||
val fileGroup = getWritePartitionPaths(metadataList).flatMap(partitionPath =>
|
||||
fsView.getAllFileGroups(partitionPath).iterator()
|
||||
).toList
|
||||
val latestCommit = fsView.getLastInstant.get().getTimestamp
|
||||
if (log.isDebugEnabled) {
|
||||
fileGroup.foreach(f => log.debug(s"current file group id: " +
|
||||
s"${f.getFileGroupId} and file slices ${f.getLatestFileSlice.get().toString}"))
|
||||
}
|
||||
|
||||
// Filter files based on user defined glob pattern
|
||||
val pathGlobPattern = optParams.getOrElse(
|
||||
DataSourceReadOptions.INCR_PATH_GLOB.key,
|
||||
DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)
|
||||
val filteredFileGroup = if(!pathGlobPattern
|
||||
.equals(DataSourceReadOptions.INCR_PATH_GLOB.defaultValue)) {
|
||||
val globMatcher = new GlobPattern("*" + pathGlobPattern)
|
||||
fileGroup.filter(f => {
|
||||
if (f.getLatestFileSlice.get().getBaseFile.isPresent) {
|
||||
globMatcher.matches(f.getLatestFileSlice.get().getBaseFile.get.getPath)
|
||||
} else {
|
||||
globMatcher.matches(f.getLatestFileSlice.get().getLatestLogFile.get().getPath.toString)
|
||||
}
|
||||
})
|
||||
} else {
|
||||
fileGroup
|
||||
}
|
||||
|
||||
// Build HoodieMergeOnReadFileSplit.
|
||||
filteredFileGroup.map(f => {
|
||||
// Ensure get the base file when there is a pending compaction, which means the base file
|
||||
// won't be in the latest file slice.
|
||||
val baseFiles = f.getAllFileSlices.iterator().filter(slice => slice.getBaseFile.isPresent).toList
|
||||
val partitionedFile = if (baseFiles.nonEmpty) {
|
||||
val baseFile = baseFiles.head.getBaseFile
|
||||
val filePath = MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath)
|
||||
Option(PartitionedFile(InternalRow.empty, filePath, 0, baseFile.get.getFileLen))
|
||||
}
|
||||
else {
|
||||
Option.empty
|
||||
}
|
||||
|
||||
val logPath = if (f.getLatestFileSlice.isPresent) {
|
||||
//If log path doesn't exist, we still include an empty path to avoid using
|
||||
// the default parquet reader to ensure the push down filter will be applied.
|
||||
Option(f.getLatestFileSlice.get().getLogFiles.iterator().toList
|
||||
.map(logfile => logfile.getPath.toString))
|
||||
}
|
||||
else {
|
||||
Option.empty
|
||||
}
|
||||
|
||||
HoodieMergeOnReadFileSplit(partitionedFile, logPath,
|
||||
latestCommit, metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType)
|
||||
})
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,263 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.HoodieLogFile
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeInputFormatUtils
|
||||
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.mapred.JobConf
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile}
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
|
||||
import org.apache.spark.sql.{Row, SQLContext}
|
||||
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
case class HoodieMergeOnReadFileSplit(dataFile: Option[PartitionedFile],
|
||||
logPaths: Option[List[String]],
|
||||
latestCommit: String,
|
||||
tablePath: String,
|
||||
maxCompactionMemoryInBytes: Long,
|
||||
mergeType: String)
|
||||
|
||||
case class HoodieMergeOnReadTableState(tableStructSchema: StructType,
|
||||
requiredStructSchema: StructType,
|
||||
tableAvroSchema: String,
|
||||
requiredAvroSchema: String,
|
||||
hoodieRealtimeFileSplits: List[HoodieMergeOnReadFileSplit],
|
||||
preCombineField: Option[String],
|
||||
recordKeyFieldOpt: Option[String])
|
||||
|
||||
class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
|
||||
val optParams: Map[String, String],
|
||||
val userSchema: StructType,
|
||||
val globPaths: Option[Seq[Path]],
|
||||
val metaClient: HoodieTableMetaClient)
|
||||
extends BaseRelation with PrunedFilteredScan with Logging {
|
||||
|
||||
private val conf = sqlContext.sparkContext.hadoopConfiguration
|
||||
private val jobConf = new JobConf(conf)
|
||||
// use schema from latest metadata, if not present, read schema from the data file
|
||||
private val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
private lazy val tableAvroSchema = {
|
||||
try {
|
||||
schemaUtil.getTableAvroSchema
|
||||
} catch {
|
||||
case _: Throwable => // If there is no commit in the table, we cann't get the schema
|
||||
// with schemaUtil, use the userSchema instead.
|
||||
SchemaConverters.toAvroType(userSchema)
|
||||
}
|
||||
}
|
||||
|
||||
private lazy val tableStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
|
||||
private val mergeType = optParams.getOrElse(
|
||||
DataSourceReadOptions.REALTIME_MERGE.key,
|
||||
DataSourceReadOptions.REALTIME_MERGE.defaultValue)
|
||||
private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf)
|
||||
private val preCombineField = {
|
||||
val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField
|
||||
if (preCombineFieldFromTableConfig != null) {
|
||||
Some(preCombineFieldFromTableConfig)
|
||||
} else {
|
||||
// get preCombineFiled from the options if this is a old table which have not store
|
||||
// the field to hoodie.properties
|
||||
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
|
||||
}
|
||||
}
|
||||
private var recordKeyFieldOpt = Option.empty[String]
|
||||
if (!metaClient.getTableConfig.populateMetaFields()) {
|
||||
recordKeyFieldOpt = Option(metaClient.getTableConfig.getRecordKeyFieldProp)
|
||||
}
|
||||
override def schema: StructType = tableStructSchema
|
||||
|
||||
override def needConversion: Boolean = false
|
||||
|
||||
private val specifiedQueryInstant = optParams.get(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key)
|
||||
.map(HoodieSqlCommonUtils.formatQueryInstant)
|
||||
|
||||
override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
|
||||
log.debug(s" buildScan requiredColumns = ${requiredColumns.mkString(",")}")
|
||||
log.debug(s" buildScan filters = ${filters.mkString(",")}")
|
||||
|
||||
val (requiredAvroSchema, requiredStructSchema) =
|
||||
MergeOnReadSnapshotRelation.getRequiredSchema(tableAvroSchema, requiredColumns)
|
||||
val fileIndex = buildFileIndex(filters)
|
||||
val hoodieTableState = HoodieMergeOnReadTableState(
|
||||
tableStructSchema,
|
||||
requiredStructSchema,
|
||||
tableAvroSchema.toString,
|
||||
requiredAvroSchema.toString,
|
||||
fileIndex,
|
||||
preCombineField,
|
||||
recordKeyFieldOpt
|
||||
)
|
||||
val fullSchemaParquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
dataSchema = tableStructSchema,
|
||||
partitionSchema = StructType(Nil),
|
||||
requiredSchema = tableStructSchema,
|
||||
filters = Seq.empty,
|
||||
options = optParams,
|
||||
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
)
|
||||
val requiredSchemaParquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
dataSchema = tableStructSchema,
|
||||
partitionSchema = StructType(Nil),
|
||||
requiredSchema = requiredStructSchema,
|
||||
filters = filters,
|
||||
options = optParams,
|
||||
hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
)
|
||||
|
||||
val rdd = new HoodieMergeOnReadRDD(
|
||||
sqlContext.sparkContext,
|
||||
jobConf,
|
||||
fullSchemaParquetReader,
|
||||
requiredSchemaParquetReader,
|
||||
hoodieTableState
|
||||
)
|
||||
rdd.asInstanceOf[RDD[Row]]
|
||||
}
|
||||
|
||||
def buildFileIndex(filters: Array[Filter]): List[HoodieMergeOnReadFileSplit] = {
|
||||
if (globPaths.isDefined) {
|
||||
// Load files from the global paths if it has defined to be compatible with the original mode
|
||||
val inMemoryFileIndex = HoodieSparkUtils.createInMemoryFileIndex(sqlContext.sparkSession, globPaths.get)
|
||||
val fsView = new HoodieTableFileSystemView(metaClient,
|
||||
// file-slice after pending compaction-requested instant-time is also considered valid
|
||||
metaClient.getCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants,
|
||||
inMemoryFileIndex.allFiles().toArray)
|
||||
val partitionPaths = fsView.getLatestBaseFiles.iterator().asScala.toList.map(_.getFileStatus.getPath.getParent)
|
||||
|
||||
|
||||
if (partitionPaths.isEmpty) { // If this an empty table, return an empty split list.
|
||||
List.empty[HoodieMergeOnReadFileSplit]
|
||||
} else {
|
||||
val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants.lastInstant()
|
||||
if (!lastInstant.isPresent) { // Return empty list if the table has no commit
|
||||
List.empty
|
||||
} else {
|
||||
val queryInstant = specifiedQueryInstant.getOrElse(lastInstant.get().getTimestamp)
|
||||
val baseAndLogsList = HoodieRealtimeInputFormatUtils.groupLogsByBaseFile(conf, partitionPaths.asJava).asScala
|
||||
val fileSplits = baseAndLogsList.map(kv => {
|
||||
val baseFile = kv.getLeft
|
||||
val logPaths = if (kv.getRight.isEmpty) Option.empty else Option(kv.getRight.asScala.toList)
|
||||
|
||||
val baseDataPath = if (baseFile.isPresent) {
|
||||
Some(PartitionedFile(
|
||||
InternalRow.empty,
|
||||
MergeOnReadSnapshotRelation.getFilePath(baseFile.get.getFileStatus.getPath),
|
||||
0, baseFile.get.getFileLen)
|
||||
)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
HoodieMergeOnReadFileSplit(baseDataPath, logPaths, queryInstant,
|
||||
metaClient.getBasePath, maxCompactionMemoryInBytes, mergeType)
|
||||
}).toList
|
||||
fileSplits
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// Load files by the HoodieFileIndex.
|
||||
val hoodieFileIndex = HoodieFileIndex(sqlContext.sparkSession, metaClient,
|
||||
Some(tableStructSchema), optParams, FileStatusCache.getOrCreate(sqlContext.sparkSession))
|
||||
|
||||
// Get partition filter and convert to catalyst expression
|
||||
val partitionColumns = hoodieFileIndex.partitionSchema.fieldNames.toSet
|
||||
val partitionFilters = filters.filter(f => f.references.forall(p => partitionColumns.contains(p)))
|
||||
val partitionFilterExpression =
|
||||
HoodieSparkUtils.convertToCatalystExpressions(partitionFilters, tableStructSchema)
|
||||
|
||||
// If convert success to catalyst expression, use the partition prune
|
||||
val fileSlices = if (partitionFilterExpression.isDefined) {
|
||||
hoodieFileIndex.listFileSlices(Seq(partitionFilterExpression.get), Seq.empty)
|
||||
} else {
|
||||
hoodieFileIndex.listFileSlices(Seq.empty, Seq.empty)
|
||||
}
|
||||
|
||||
if (fileSlices.isEmpty) {
|
||||
// If this an empty table, return an empty split list.
|
||||
List.empty[HoodieMergeOnReadFileSplit]
|
||||
} else {
|
||||
val fileSplits = fileSlices.values.flatten.map(fileSlice => {
|
||||
val latestInstant = metaClient.getActiveTimeline.getCommitsTimeline
|
||||
.filterCompletedInstants.lastInstant().get().getTimestamp
|
||||
val queryInstant = specifiedQueryInstant.getOrElse(latestInstant)
|
||||
|
||||
val partitionedFile = if (fileSlice.getBaseFile.isPresent) {
|
||||
val baseFile = fileSlice.getBaseFile.get()
|
||||
val baseFilePath = MergeOnReadSnapshotRelation.getFilePath(baseFile.getFileStatus.getPath)
|
||||
Option(PartitionedFile(InternalRow.empty, baseFilePath, 0, baseFile.getFileLen))
|
||||
} else {
|
||||
Option.empty
|
||||
}
|
||||
|
||||
val logPaths = fileSlice.getLogFiles.sorted(HoodieLogFile.getLogFileComparator).iterator().asScala
|
||||
.map(logFile => MergeOnReadSnapshotRelation.getFilePath(logFile.getPath)).toList
|
||||
val logPathsOptional = if (logPaths.isEmpty) Option.empty else Option(logPaths)
|
||||
HoodieMergeOnReadFileSplit(partitionedFile, logPathsOptional, queryInstant, metaClient.getBasePath,
|
||||
maxCompactionMemoryInBytes, mergeType)
|
||||
}).toList
|
||||
fileSplits
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object MergeOnReadSnapshotRelation {
|
||||
|
||||
def getFilePath(path: Path): String = {
|
||||
// Here we use the Path#toUri to encode the path string, as there is a decode in
|
||||
// ParquetFileFormat#buildReaderWithPartitionValues in the spark project when read the table
|
||||
// .So we should encode the file path here. Otherwise, there is a FileNotException throw
|
||||
// out.
|
||||
// For example, If the "pt" is the partition path field, and "pt" = "2021/02/02", If
|
||||
// we enable the URL_ENCODE_PARTITIONING and write data to hudi table.The data path
|
||||
// in the table will just like "/basePath/2021%2F02%2F02/xxxx.parquet". When we read
|
||||
// data from the table, if there are no encode for the file path,
|
||||
// ParquetFileFormat#buildReaderWithPartitionValues will decode it to
|
||||
// "/basePath/2021/02/02/xxxx.parquet" witch will result to a FileNotException.
|
||||
// See FileSourceScanExec#createBucketedReadRDD in spark project which do the same thing
|
||||
// when create PartitionedFile.
|
||||
path.toUri.toString
|
||||
}
|
||||
|
||||
def getRequiredSchema(tableAvroSchema: Schema, requiredColumns: Array[String]): (Schema, StructType) = {
|
||||
// First get the required avro-schema, then convert the avro-schema to spark schema.
|
||||
val name2Fields = tableAvroSchema.getFields.asScala.map(f => f.name() -> f).toMap
|
||||
val requiredFields = requiredColumns.map(c => name2Fields(c))
|
||||
.map(f => new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order())).toList
|
||||
val requiredAvroSchema = Schema.createRecord(tableAvroSchema.getName, tableAvroSchema.getDoc,
|
||||
tableAvroSchema.getNamespace, tableAvroSchema.isError, requiredFields.asJava)
|
||||
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
|
||||
(requiredAvroSchema, requiredStructSchema)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.spark.sql.{DataFrame, DataFrameReader, DataFrameWriter}
|
||||
|
||||
package object hudi {
|
||||
|
||||
/**
|
||||
* Adds a method, `hoodie`, to DataFrameWriter
|
||||
*/
|
||||
implicit class AvroDataFrameWriter[T](writer: DataFrameWriter[T]) {
|
||||
def avro: String => Unit = writer.format("org.apache.hudi").save
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a method, `hoodie`, to DataFrameReader
|
||||
*/
|
||||
implicit class AvroDataFrameReader(reader: DataFrameReader) {
|
||||
def avro: String => DataFrame = reader.format("org.apache.hudi").load
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* 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.avro
|
||||
|
||||
import org.apache.avro.Schema
|
||||
|
||||
import org.apache.hudi.HoodieSparkUtils
|
||||
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
/**
|
||||
* This is to be compatible with the type returned by Spark 3.1
|
||||
* and other spark versions for AvroDeserializer
|
||||
*/
|
||||
case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) {
|
||||
|
||||
private val avroDeserializer = if (HoodieSparkUtils.isSpark3_2) {
|
||||
// SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments.
|
||||
// So use the reflection to get AvroDeserializer instance.
|
||||
val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType], classOf[String])
|
||||
constructor.newInstance(rootAvroType, rootCatalystType, "EXCEPTION")
|
||||
} else {
|
||||
val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType])
|
||||
constructor.newInstance(rootAvroType, rootCatalystType)
|
||||
}
|
||||
|
||||
def deserializeData(data: Any): Any = {
|
||||
avroDeserializer.deserialize(data) match {
|
||||
case Some(r) => r // As of spark 3.1, this will return data wrapped with Option, so we fetch the data.
|
||||
case o => o // for other spark version, return the data directly.
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,28 @@
|
||||
/*
|
||||
* 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.avro
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.spark.sql.types.DataType
|
||||
|
||||
/**
|
||||
* As AvroSerializer cannot be access out of the spark.sql.avro package since spark 3.1, we define
|
||||
* this class to be accessed by other class.
|
||||
*/
|
||||
case class HoodieAvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean)
|
||||
extends AvroSerializer(rootCatalystType, rootAvroType, nullable)
|
||||
@@ -0,0 +1,308 @@
|
||||
/*
|
||||
* 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.catalyst.catalog
|
||||
|
||||
import org.apache.hudi.HoodieWriterUtils._
|
||||
import org.apache.hudi.common.config.DFSPropertiesConfiguration
|
||||
import org.apache.hudi.common.model.HoodieTableType
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.util.ValidationUtils
|
||||
import org.apache.hudi.keygen.ComplexKeyGenerator
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
|
||||
import org.apache.spark.sql.types.{StructField, StructType}
|
||||
|
||||
import java.util.{Locale, Properties}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
* A wrapper of hoodie CatalogTable instance and hoodie Table.
|
||||
*/
|
||||
class HoodieCatalogTable(val spark: SparkSession, val table: CatalogTable) extends Logging {
|
||||
|
||||
assert(table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi", "It's not a Hudi table")
|
||||
|
||||
private val hadoopConf = spark.sessionState.newHadoopConf
|
||||
|
||||
/**
|
||||
* database.table in catalog
|
||||
*/
|
||||
val catalogTableName = table.qualifiedName
|
||||
|
||||
/**
|
||||
* properties defined in catalog.
|
||||
*/
|
||||
val catalogProperties: Map[String, String] = table.storage.properties ++ table.properties
|
||||
|
||||
/**
|
||||
* hoodie table's location.
|
||||
* if create managed hoodie table, use `catalog.defaultTablePath`.
|
||||
*/
|
||||
val tableLocation: String = HoodieSqlCommonUtils.getTableLocation(table, spark)
|
||||
|
||||
/**
|
||||
* A flag to whether the hoodie table exists.
|
||||
*/
|
||||
val hoodieTableExists: Boolean = tableExistsInPath(tableLocation, hadoopConf)
|
||||
|
||||
/**
|
||||
* Meta Client.
|
||||
*/
|
||||
lazy val metaClient: HoodieTableMetaClient = HoodieTableMetaClient.builder()
|
||||
.setBasePath(tableLocation)
|
||||
.setConf(hadoopConf)
|
||||
.build()
|
||||
|
||||
/**
|
||||
* Hoodie Table Config
|
||||
*/
|
||||
lazy val tableConfig: HoodieTableConfig = metaClient.getTableConfig
|
||||
|
||||
/**
|
||||
* the name of table
|
||||
*/
|
||||
lazy val tableName: String = tableConfig.getTableName
|
||||
|
||||
/**
|
||||
* The name of type of table
|
||||
*/
|
||||
lazy val tableType: HoodieTableType = tableConfig.getTableType
|
||||
|
||||
/**
|
||||
* The type of table
|
||||
*/
|
||||
lazy val tableTypeName: String = tableType.name()
|
||||
|
||||
/**
|
||||
* Record Field List(Primary Key List)
|
||||
*/
|
||||
lazy val primaryKeys: Array[String] = tableConfig.getRecordKeyFields.orElse(Array.empty)
|
||||
|
||||
/**
|
||||
* PreCombine Field
|
||||
*/
|
||||
lazy val preCombineKey: Option[String] = Option(tableConfig.getPreCombineField)
|
||||
|
||||
/**
|
||||
* Partition Fields
|
||||
*/
|
||||
lazy val partitionFields: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty)
|
||||
|
||||
/**
|
||||
* The schema of table.
|
||||
* Make StructField nullable.
|
||||
*/
|
||||
lazy val tableSchema: StructType = {
|
||||
val originSchema = getTableSqlSchema(metaClient, includeMetadataFields = true).getOrElse(table.schema)
|
||||
StructType(originSchema.map(_.copy(nullable = true)))
|
||||
}
|
||||
|
||||
/**
|
||||
* The schema without hoodie meta fields
|
||||
*/
|
||||
lazy val tableSchemaWithoutMetaFields: StructType = HoodieSqlCommonUtils.removeMetaFields(tableSchema)
|
||||
|
||||
/**
|
||||
* The schema of data fields
|
||||
*/
|
||||
lazy val dataSchema: StructType = {
|
||||
StructType(tableSchema.filterNot(f => partitionFields.contains(f.name)))
|
||||
}
|
||||
|
||||
/**
|
||||
* The schema of data fields not including hoodie meta fields
|
||||
*/
|
||||
lazy val dataSchemaWithoutMetaFields: StructType = HoodieSqlCommonUtils.removeMetaFields(dataSchema)
|
||||
|
||||
/**
|
||||
* The schema of partition fields
|
||||
*/
|
||||
lazy val partitionSchema: StructType = StructType(tableSchema.filter(f => partitionFields.contains(f.name)))
|
||||
|
||||
/**
|
||||
* All the partition paths
|
||||
*/
|
||||
def getAllPartitionPaths: Seq[String] = HoodieSqlCommonUtils.getAllPartitionPaths(spark, table)
|
||||
|
||||
/**
|
||||
* Check if table is a partitioned table
|
||||
*/
|
||||
def isPartitionedTable: Boolean = table.partitionColumnNames.nonEmpty
|
||||
|
||||
/**
|
||||
* init hoodie table for create table (as select)
|
||||
*/
|
||||
def initHoodieTable(): Unit = {
|
||||
logInfo(s"Init hoodie.properties for ${table.identifier.unquotedString}")
|
||||
val (finalSchema, tableConfigs) = parseSchemaAndConfigs()
|
||||
|
||||
// Save all the table config to the hoodie.properties.
|
||||
val properties = new Properties()
|
||||
properties.putAll(tableConfigs.asJava)
|
||||
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(properties)
|
||||
.setTableName(table.identifier.table)
|
||||
.setTableCreateSchema(SchemaConverters.toAvroType(finalSchema).toString())
|
||||
.setPartitionFields(table.partitionColumnNames.mkString(","))
|
||||
.initTable(hadoopConf, tableLocation)
|
||||
}
|
||||
|
||||
/**
|
||||
* @return schema, table parameters in which all parameters aren't sql-styled.
|
||||
*/
|
||||
private def parseSchemaAndConfigs(): (StructType, Map[String, String]) = {
|
||||
val globalProps = DFSPropertiesConfiguration.getGlobalProps.asScala.toMap
|
||||
val globalTableConfigs = mappingSparkDatasourceConfigsToTableConfigs(globalProps)
|
||||
val globalSqlOptions = HoodieOptionConfig.mappingTableConfigToSqlOption(globalTableConfigs)
|
||||
|
||||
val sqlOptions = HoodieOptionConfig.withDefaultSqlOptions(globalSqlOptions ++ catalogProperties)
|
||||
|
||||
// get final schema and parameters
|
||||
val (finalSchema, tableConfigs) = (table.tableType, hoodieTableExists) match {
|
||||
case (CatalogTableType.EXTERNAL, true) =>
|
||||
val existingTableConfig = tableConfig.getProps.asScala.toMap
|
||||
val currentTableConfig = globalTableConfigs ++ existingTableConfig
|
||||
val catalogTableProps = HoodieOptionConfig.mappingSqlOptionToTableConfig(catalogProperties)
|
||||
validateTableConfig(spark, catalogTableProps, convertMapToHoodieConfig(existingTableConfig))
|
||||
|
||||
val options = extraTableConfig(spark, hoodieTableExists, currentTableConfig) ++
|
||||
HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions) ++ currentTableConfig
|
||||
|
||||
ValidationUtils.checkArgument(tableSchema.nonEmpty || table.schema.nonEmpty,
|
||||
s"Missing schema for Create Table: $catalogTableName")
|
||||
val schema = if (tableSchema.nonEmpty) {
|
||||
tableSchema
|
||||
} else {
|
||||
addMetaFields(table.schema)
|
||||
}
|
||||
|
||||
(schema, options)
|
||||
|
||||
case (_, false) =>
|
||||
ValidationUtils.checkArgument(table.schema.nonEmpty,
|
||||
s"Missing schema for Create Table: $catalogTableName")
|
||||
val schema = table.schema
|
||||
val options = extraTableConfig(spark, isTableExists = false, globalTableConfigs) ++
|
||||
HoodieOptionConfig.mappingSqlOptionToTableConfig(sqlOptions)
|
||||
(addMetaFields(schema), options)
|
||||
|
||||
case (CatalogTableType.MANAGED, true) =>
|
||||
throw new AnalysisException(s"Can not create the managed table('$catalogTableName')" +
|
||||
s". The associated location('$tableLocation') already exists.")
|
||||
}
|
||||
HoodieOptionConfig.validateTable(spark, finalSchema,
|
||||
HoodieOptionConfig.mappingTableConfigToSqlOption(tableConfigs))
|
||||
|
||||
val resolver = spark.sessionState.conf.resolver
|
||||
val dataSchema = finalSchema.filterNot { f =>
|
||||
table.partitionColumnNames.exists(resolver(_, f.name))
|
||||
}
|
||||
verifyDataSchema(table.identifier, table.tableType, dataSchema)
|
||||
|
||||
(finalSchema, tableConfigs)
|
||||
}
|
||||
|
||||
private def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean,
|
||||
originTableConfig: Map[String, String] = Map.empty): Map[String, String] = {
|
||||
val extraConfig = mutable.Map.empty[String, String]
|
||||
if (isTableExists) {
|
||||
val allPartitionPaths = getAllPartitionPaths
|
||||
if (originTableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)) {
|
||||
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) =
|
||||
originTableConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
|
||||
} else {
|
||||
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) =
|
||||
String.valueOf(isHiveStyledPartitioning(allPartitionPaths, table))
|
||||
}
|
||||
if (originTableConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)) {
|
||||
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) =
|
||||
originTableConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
|
||||
} else {
|
||||
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) =
|
||||
String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table))
|
||||
}
|
||||
} else {
|
||||
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true"
|
||||
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue()
|
||||
}
|
||||
|
||||
if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) {
|
||||
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) =
|
||||
HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(
|
||||
originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key))
|
||||
} else {
|
||||
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[ComplexKeyGenerator].getCanonicalName
|
||||
}
|
||||
extraConfig.toMap
|
||||
}
|
||||
|
||||
// This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#verifyDataSchema
|
||||
private def verifyDataSchema(tableIdentifier: TableIdentifier, tableType: CatalogTableType,
|
||||
dataSchema: Seq[StructField]): Unit = {
|
||||
if (tableType != CatalogTableType.VIEW) {
|
||||
val invalidChars = Seq(",", ":", ";")
|
||||
def verifyNestedColumnNames(schema: StructType): Unit = schema.foreach { f =>
|
||||
f.dataType match {
|
||||
case st: StructType => verifyNestedColumnNames(st)
|
||||
case _ if invalidChars.exists(f.name.contains) =>
|
||||
val invalidCharsString = invalidChars.map(c => s"'$c'").mkString(", ")
|
||||
val errMsg = "Cannot create a table having a nested column whose name contains " +
|
||||
s"invalid characters ($invalidCharsString) in Hive metastore. Table: $tableIdentifier; " +
|
||||
s"Column: ${f.name}"
|
||||
throw new AnalysisException(errMsg)
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
|
||||
dataSchema.foreach { f =>
|
||||
f.dataType match {
|
||||
// Checks top-level column names
|
||||
case _ if f.name.contains(",") =>
|
||||
throw new AnalysisException("Cannot create a table having a column whose name " +
|
||||
s"contains commas in Hive metastore. Table: $tableIdentifier; Column: ${f.name}")
|
||||
// Checks nested column names
|
||||
case st: StructType =>
|
||||
verifyNestedColumnNames(st)
|
||||
case _ =>
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object HoodieCatalogTable {
|
||||
|
||||
def apply(sparkSession: SparkSession, tableIdentifier: TableIdentifier): HoodieCatalogTable = {
|
||||
val catalogTable = sparkSession.sessionState.catalog.getTableMetadata(tableIdentifier)
|
||||
HoodieCatalogTable(sparkSession, catalogTable)
|
||||
}
|
||||
|
||||
def apply(sparkSession: SparkSession, catalogTable: CatalogTable): HoodieCatalogTable = {
|
||||
new HoodieCatalogTable(sparkSession, catalogTable)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* 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.catalyst.trees
|
||||
|
||||
/**
|
||||
* Similar to `LeafLike` in Spark3.2.
|
||||
*/
|
||||
trait HoodieLeafLike[T <: TreeNode[T]] { self: TreeNode[T] =>
|
||||
|
||||
override final def children: Seq[T] = Nil
|
||||
|
||||
override final def mapChildren(f: T => T): T = this.asInstanceOf[T]
|
||||
|
||||
final def withNewChildrenInternal(newChildren: IndexedSeq[T]): T = this.asInstanceOf[T]
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.hive
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.sql.hive.client.HiveClient
|
||||
|
||||
object HiveClientUtils {
|
||||
|
||||
def newClientForMetadata(conf: SparkConf, hadoopConf: Configuration): HiveClient = {
|
||||
HiveUtils.newClientForMetadata(conf, hadoopConf)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,292 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hudi.index.columnstats.ColumnStatsIndexHelper.{getMaxColumnNameFor, getMinColumnNameFor, getNumNullsColumnNameFor}
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
|
||||
import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
|
||||
import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, EqualNullSafe, EqualTo, Expression, ExtractValue, GetStructField, GreaterThan, GreaterThanOrEqual, In, IsNotNull, IsNull, LessThan, LessThanOrEqual, Literal, Not, Or, StartsWith}
|
||||
import org.apache.spark.sql.execution.datasources.PartitionedFile
|
||||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
|
||||
import org.apache.spark.sql.functions.col
|
||||
import org.apache.spark.sql.sources.Filter
|
||||
import org.apache.spark.sql.types.{StringType, StructType}
|
||||
import org.apache.spark.sql.vectorized.ColumnarBatch
|
||||
import org.apache.spark.sql.{AnalysisException, SparkSession}
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
object DataSkippingUtils extends Logging {
|
||||
|
||||
/**
|
||||
* Translates provided {@link filterExpr} into corresponding filter-expression for column-stats index index table
|
||||
* to filter out candidate files that would hold records matching the original filter
|
||||
*
|
||||
* @param sourceFilterExpr source table's query's filter expression
|
||||
* @param indexSchema index table schema
|
||||
* @return filter for column-stats index's table
|
||||
*/
|
||||
def createColumnStatsIndexFilterExpr(sourceFilterExpr: Expression, indexSchema: StructType): Expression = {
|
||||
// Try to transform original Source Table's filter expression into
|
||||
// Column-Stats Index filter expression
|
||||
tryComposeIndexFilterExpr(sourceFilterExpr, indexSchema) match {
|
||||
case Some(e) => e
|
||||
// NOTE: In case we can't transform source filter expression, we fallback
|
||||
// to {@code TrueLiteral}, to essentially avoid pruning any indexed files from scanning
|
||||
case None => TrueLiteral
|
||||
}
|
||||
}
|
||||
|
||||
private def tryComposeIndexFilterExpr(sourceExpr: Expression, indexSchema: StructType): Option[Expression] = {
|
||||
def minValue(colName: String) = col(getMinColumnNameFor(colName)).expr
|
||||
def maxValue(colName: String) = col(getMaxColumnNameFor(colName)).expr
|
||||
def numNulls(colName: String) = col(getNumNullsColumnNameFor(colName)).expr
|
||||
|
||||
def colContainsValuesEqualToLiteral(colName: String, value: Literal): Expression =
|
||||
// Only case when column C contains value V is when min(C) <= V <= max(c)
|
||||
And(LessThanOrEqual(minValue(colName), value), GreaterThanOrEqual(maxValue(colName), value))
|
||||
|
||||
def colContainsOnlyValuesEqualToLiteral(colName: String, value: Literal) =
|
||||
// Only case when column C contains _only_ value V is when min(C) = V AND max(c) = V
|
||||
And(EqualTo(minValue(colName), value), EqualTo(maxValue(colName), value))
|
||||
|
||||
sourceExpr match {
|
||||
// Filter "colA = b"
|
||||
// Translates to "colA_minValue <= b AND colA_maxValue >= b" condition for index lookup
|
||||
case EqualTo(attribute: AttributeReference, value: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => colContainsValuesEqualToLiteral(colName, value))
|
||||
|
||||
// Filter "b = colA"
|
||||
// Translates to "colA_minValue <= b AND colA_maxValue >= b" condition for index lookup
|
||||
case EqualTo(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => colContainsValuesEqualToLiteral(colName, value))
|
||||
|
||||
// Filter "colA != b"
|
||||
// Translates to "NOT(colA_minValue = b AND colA_maxValue = b)"
|
||||
// NOTE: This is NOT an inversion of `colA = b`
|
||||
case Not(EqualTo(attribute: AttributeReference, value: Literal)) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => Not(colContainsOnlyValuesEqualToLiteral(colName, value)))
|
||||
|
||||
// Filter "b != colA"
|
||||
// Translates to "NOT(colA_minValue = b AND colA_maxValue = b)"
|
||||
// NOTE: This is NOT an inversion of `colA = b`
|
||||
case Not(EqualTo(value: Literal, attribute: AttributeReference)) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => Not(colContainsOnlyValuesEqualToLiteral(colName, value)))
|
||||
|
||||
// Filter "colA = null"
|
||||
// Translates to "colA_num_nulls = null" for index lookup
|
||||
case equalNullSafe @ EqualNullSafe(_: AttributeReference, _ @ Literal(null, _)) =>
|
||||
getTargetIndexedColName(equalNullSafe.left, indexSchema)
|
||||
.map(colName => EqualTo(numNulls(colName), equalNullSafe.right))
|
||||
|
||||
// Filter "colA < b"
|
||||
// Translates to "colA_minValue < b" for index lookup
|
||||
case LessThan(attribute: AttributeReference, value: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThan(minValue(colName), value))
|
||||
|
||||
// Filter "b > colA"
|
||||
// Translates to "b > colA_minValue" for index lookup
|
||||
case GreaterThan(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThan(minValue(colName), value))
|
||||
|
||||
// Filter "b < colA"
|
||||
// Translates to "b < colA_maxValue" for index lookup
|
||||
case LessThan(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThan(maxValue(colName), value))
|
||||
|
||||
// Filter "colA > b"
|
||||
// Translates to "colA_maxValue > b" for index lookup
|
||||
case GreaterThan(attribute: AttributeReference, value: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThan(maxValue(colName), value))
|
||||
|
||||
// Filter "colA <= b"
|
||||
// Translates to "colA_minValue <= b" for index lookup
|
||||
case LessThanOrEqual(attribute: AttributeReference, value: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThanOrEqual(minValue(colName), value))
|
||||
|
||||
// Filter "b >= colA"
|
||||
// Translates to "b >= colA_minValue" for index lookup
|
||||
case GreaterThanOrEqual(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => LessThanOrEqual(minValue(colName), value))
|
||||
|
||||
// Filter "b <= colA"
|
||||
// Translates to "b <= colA_maxValue" for index lookup
|
||||
case LessThanOrEqual(value: Literal, attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThanOrEqual(maxValue(colName), value))
|
||||
|
||||
// Filter "colA >= b"
|
||||
// Translates to "colA_maxValue >= b" for index lookup
|
||||
case GreaterThanOrEqual(attribute: AttributeReference, right: Literal) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThanOrEqual(maxValue(colName), right))
|
||||
|
||||
// Filter "colA is null"
|
||||
// Translates to "colA_num_nulls > 0" for index lookup
|
||||
case IsNull(attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => GreaterThan(numNulls(colName), Literal(0)))
|
||||
|
||||
// Filter "colA is not null"
|
||||
// Translates to "colA_num_nulls = 0" for index lookup
|
||||
case IsNotNull(attribute: AttributeReference) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => EqualTo(numNulls(colName), Literal(0)))
|
||||
|
||||
// Filter "colA in (a, b, ...)"
|
||||
// Translates to "(colA_minValue <= a AND colA_maxValue >= a) OR (colA_minValue <= b AND colA_maxValue >= b)" for index lookup
|
||||
// NOTE: This is equivalent to "colA = a OR colA = b OR ..."
|
||||
case In(attribute: AttributeReference, list: Seq[Literal]) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName =>
|
||||
list.map { lit => colContainsValuesEqualToLiteral(colName, lit) }.reduce(Or)
|
||||
)
|
||||
|
||||
// Filter "colA not in (a, b, ...)"
|
||||
// Translates to "NOT((colA_minValue = a AND colA_maxValue = a) OR (colA_minValue = b AND colA_maxValue = b))" for index lookup
|
||||
// NOTE: This is NOT an inversion of `in (a, b, ...)` expr, this is equivalent to "colA != a AND colA != b AND ..."
|
||||
case Not(In(attribute: AttributeReference, list: Seq[Literal])) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName =>
|
||||
Not(
|
||||
list.map { lit => colContainsOnlyValuesEqualToLiteral(colName, lit) }.reduce(Or)
|
||||
)
|
||||
)
|
||||
|
||||
// Filter "colA like 'xxx%'"
|
||||
// Translates to "colA_minValue <= xxx AND colA_maxValue >= xxx" for index lookup
|
||||
// NOTE: That this operator only matches string prefixes, and this is
|
||||
// essentially equivalent to "colA = b" expression
|
||||
case StartsWith(attribute, v @ Literal(_: UTF8String, _)) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName => colContainsValuesEqualToLiteral(colName, v))
|
||||
|
||||
// Filter "colA not like 'xxx%'"
|
||||
// Translates to "NOT(colA_minValue like 'xxx%' AND colA_maxValue like 'xxx%')" for index lookup
|
||||
// NOTE: This is NOT an inversion of "colA like xxx"
|
||||
case Not(StartsWith(attribute, value @ Literal(_: UTF8String, _))) =>
|
||||
getTargetIndexedColName(attribute, indexSchema)
|
||||
.map(colName =>
|
||||
Not(And(StartsWith(minValue(colName), value), StartsWith(maxValue(colName), value)))
|
||||
)
|
||||
|
||||
case or: Or =>
|
||||
val resLeft = createColumnStatsIndexFilterExpr(or.left, indexSchema)
|
||||
val resRight = createColumnStatsIndexFilterExpr(or.right, indexSchema)
|
||||
|
||||
Option(Or(resLeft, resRight))
|
||||
|
||||
case and: And =>
|
||||
val resLeft = createColumnStatsIndexFilterExpr(and.left, indexSchema)
|
||||
val resRight = createColumnStatsIndexFilterExpr(and.right, indexSchema)
|
||||
|
||||
Option(And(resLeft, resRight))
|
||||
|
||||
//
|
||||
// Pushing Logical NOT inside the AND/OR expressions
|
||||
// NOTE: This is required to make sure we're properly handling negations in
|
||||
// cases like {@code NOT(colA = 0)}, {@code NOT(colA in (a, b, ...)}
|
||||
//
|
||||
|
||||
case Not(And(left: Expression, right: Expression)) =>
|
||||
Option(createColumnStatsIndexFilterExpr(Or(Not(left), Not(right)), indexSchema))
|
||||
|
||||
case Not(Or(left: Expression, right: Expression)) =>
|
||||
Option(createColumnStatsIndexFilterExpr(And(Not(left), Not(right)), indexSchema))
|
||||
|
||||
case _: Expression => None
|
||||
}
|
||||
}
|
||||
|
||||
private def checkColIsIndexed(colName: String, indexSchema: StructType): Boolean = {
|
||||
Set.apply(
|
||||
getMinColumnNameFor(colName),
|
||||
getMaxColumnNameFor(colName),
|
||||
getNumNullsColumnNameFor(colName)
|
||||
)
|
||||
.forall(stat => indexSchema.exists(_.name == stat))
|
||||
}
|
||||
|
||||
private def getTargetIndexedColName(resolvedExpr: Expression, indexSchema: StructType): Option[String] = {
|
||||
val colName = UnresolvedAttribute(getTargetColNameParts(resolvedExpr)).name
|
||||
|
||||
// Verify that the column is indexed
|
||||
if (checkColIsIndexed(colName, indexSchema)) {
|
||||
Option.apply(colName)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
private def getTargetColNameParts(resolvedTargetCol: Expression): Seq[String] = {
|
||||
resolvedTargetCol match {
|
||||
case attr: Attribute => Seq(attr.name)
|
||||
case Alias(c, _) => getTargetColNameParts(c)
|
||||
case GetStructField(c, _, Some(name)) => getTargetColNameParts(c) :+ name
|
||||
case ex: ExtractValue =>
|
||||
throw new AnalysisException(s"convert reference to name failed, Updating nested fields is only supported for StructType: ${ex}.")
|
||||
case other =>
|
||||
throw new AnalysisException(s"convert reference to name failed, Found unsupported expression ${other}")
|
||||
}
|
||||
}
|
||||
|
||||
def getIndexFiles(conf: Configuration, indexPath: String): Seq[FileStatus] = {
|
||||
val basePath = new Path(indexPath)
|
||||
basePath.getFileSystem(conf)
|
||||
.listStatus(basePath).filter(f => f.getPath.getName.endsWith(".parquet"))
|
||||
}
|
||||
|
||||
/**
|
||||
* read parquet files concurrently by local.
|
||||
* this method is mush faster than spark
|
||||
*/
|
||||
def readParquetFile(spark: SparkSession, indexFiles: Seq[FileStatus], filters: Seq[Filter] = Nil, schemaOpts: Option[StructType] = None): Set[String] = {
|
||||
val hadoopConf = spark.sparkContext.hadoopConfiguration
|
||||
val partitionedFiles = indexFiles.map(f => PartitionedFile(InternalRow.empty, f.getPath.toString, 0, f.getLen))
|
||||
|
||||
val requiredSchema = new StructType().add("file", StringType, true)
|
||||
val schema = schemaOpts.getOrElse(requiredSchema)
|
||||
val parquetReader = new ParquetFileFormat().buildReaderWithPartitionValues(spark
|
||||
, schema , StructType(Nil), requiredSchema, filters, Map.empty, hadoopConf)
|
||||
val results = new Array[Iterator[String]](partitionedFiles.size)
|
||||
partitionedFiles.zipWithIndex.par.foreach { case (pf, index) =>
|
||||
val fileIterator = parquetReader(pf).asInstanceOf[Iterator[Any]]
|
||||
val rows = fileIterator.flatMap(_ match {
|
||||
case r: InternalRow => Seq(r)
|
||||
case b: ColumnarBatch => b.rowIterator().asScala
|
||||
}).map(r => r.getString(0))
|
||||
results(index) = rows
|
||||
}
|
||||
results.flatMap(f => f).toSet
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,263 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions
|
||||
import org.apache.hudi.common.model.DefaultHoodieRecordPayload
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.util.ValidationUtils
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.types.StructType
|
||||
|
||||
|
||||
/**
|
||||
* The HoodieOptionConfig defines some short name for the hoodie
|
||||
* option key and value.
|
||||
*/
|
||||
object HoodieOptionConfig {
|
||||
|
||||
/**
|
||||
* The short name for the value of COW_TABLE_TYPE_OPT_VAL.
|
||||
*/
|
||||
val SQL_VALUE_TABLE_TYPE_COW = "cow"
|
||||
|
||||
/**
|
||||
* The short name for the value of MOR_TABLE_TYPE_OPT_VAL.
|
||||
*/
|
||||
val SQL_VALUE_TABLE_TYPE_MOR = "mor"
|
||||
|
||||
|
||||
val SQL_KEY_TABLE_PRIMARY_KEY: HoodieSQLOption[String] = buildConf()
|
||||
.withSqlKey("primaryKey")
|
||||
.withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD.key)
|
||||
.withTableConfigKey(HoodieTableConfig.RECORDKEY_FIELDS.key)
|
||||
.defaultValue(DataSourceWriteOptions.RECORDKEY_FIELD.defaultValue())
|
||||
.build()
|
||||
|
||||
val SQL_KEY_TABLE_TYPE: HoodieSQLOption[String] = buildConf()
|
||||
.withSqlKey("type")
|
||||
.withHoodieKey(DataSourceWriteOptions.TABLE_TYPE.key)
|
||||
.withTableConfigKey(HoodieTableConfig.TYPE.key)
|
||||
.defaultValue(SQL_VALUE_TABLE_TYPE_COW)
|
||||
.build()
|
||||
|
||||
val SQL_KEY_PRECOMBINE_FIELD: HoodieSQLOption[String] = buildConf()
|
||||
.withSqlKey("preCombineField")
|
||||
.withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD.key)
|
||||
.withTableConfigKey(HoodieTableConfig.PRECOMBINE_FIELD.key)
|
||||
.build()
|
||||
|
||||
val SQL_PAYLOAD_CLASS: HoodieSQLOption[String] = buildConf()
|
||||
.withSqlKey("payloadClass")
|
||||
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key)
|
||||
.withTableConfigKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key)
|
||||
.defaultValue(classOf[DefaultHoodieRecordPayload].getName)
|
||||
.build()
|
||||
|
||||
/**
|
||||
* The mapping of the sql short name key to the hoodie's config key.
|
||||
*/
|
||||
private lazy val keyMapping: Map[String, String] = {
|
||||
HoodieOptionConfig.getClass.getDeclaredFields
|
||||
.filter(f => f.getType == classOf[HoodieSQLOption[_]])
|
||||
.map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]})
|
||||
.map(option => option.sqlKeyName -> option.hoodieKeyName)
|
||||
.toMap
|
||||
}
|
||||
|
||||
/**
|
||||
* The mapping of the sql short name key to the hoodie table config key
|
||||
* defined in HoodieTableConfig.
|
||||
*/
|
||||
private lazy val keyTableConfigMapping: Map[String, String] = {
|
||||
HoodieOptionConfig.getClass.getDeclaredFields
|
||||
.filter(f => f.getType == classOf[HoodieSQLOption[_]])
|
||||
.map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]})
|
||||
.filter(_.tableConfigKey.isDefined)
|
||||
.map(option => option.sqlKeyName -> option.tableConfigKey.get)
|
||||
.toMap
|
||||
}
|
||||
|
||||
private lazy val tableConfigKeyToSqlKey: Map[String, String] =
|
||||
keyTableConfigMapping.map(f => f._2 -> f._1)
|
||||
|
||||
/**
|
||||
* Mapping of the short sql value to the hoodie's config value
|
||||
*/
|
||||
private val valueMapping: Map[String, String] = Map (
|
||||
SQL_VALUE_TABLE_TYPE_COW -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
SQL_VALUE_TABLE_TYPE_MOR -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL
|
||||
)
|
||||
|
||||
private lazy val reverseValueMapping = valueMapping.map(f => f._2 -> f._1)
|
||||
|
||||
def withDefaultSqlOptions(options: Map[String, String]): Map[String, String] = defaultSqlOptions ++ options
|
||||
|
||||
/**
|
||||
* Mapping the sql's short name key/value in the options to the hoodie's config key/value.
|
||||
* @param options
|
||||
* @return
|
||||
*/
|
||||
def mappingSqlOptionToHoodieParam(options: Map[String, String]): Map[String, String] = {
|
||||
options.map (kv =>
|
||||
keyMapping.getOrElse(kv._1, kv._1) -> valueMapping.getOrElse(kv._2, kv._2))
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping the sql options to the hoodie table config which used to store to the hoodie
|
||||
* .properties when create the table.
|
||||
* @param options
|
||||
* @return
|
||||
*/
|
||||
def mappingSqlOptionToTableConfig(options: Map[String, String]): Map[String, String] = {
|
||||
options.map { case (k, v) =>
|
||||
if (keyTableConfigMapping.contains(k)) {
|
||||
keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v)
|
||||
} else {
|
||||
k -> v
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping the table config (loaded from the hoodie.properties) to the sql options.
|
||||
*/
|
||||
def mappingTableConfigToSqlOption(options: Map[String, String]): Map[String, String] = {
|
||||
options.map(kv => tableConfigKeyToSqlKey.getOrElse(kv._1, kv._1) -> reverseValueMapping.getOrElse(kv._2, kv._2))
|
||||
}
|
||||
|
||||
val defaultSqlOptions: Map[String, String] = {
|
||||
HoodieOptionConfig.getClass.getDeclaredFields
|
||||
.filter(f => f.getType == classOf[HoodieSQLOption[_]])
|
||||
.map(f => {f.setAccessible(true); f.get(HoodieOptionConfig).asInstanceOf[HoodieSQLOption[_]]})
|
||||
.filter(option => option.tableConfigKey.isDefined && option.defaultValue.isDefined)
|
||||
.map(option => option.sqlKeyName -> option.defaultValue.get.toString)
|
||||
.toMap
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the primary key from the table options.
|
||||
* @param options
|
||||
* @return
|
||||
*/
|
||||
def getPrimaryColumns(options: Map[String, String]): Array[String] = {
|
||||
val params = mappingSqlOptionToHoodieParam(options)
|
||||
params.get(DataSourceWriteOptions.RECORDKEY_FIELD.key)
|
||||
.map(_.split(",").filter(_.nonEmpty))
|
||||
.getOrElse(Array.empty)
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the table type from the table options.
|
||||
* @param options
|
||||
* @return
|
||||
*/
|
||||
def getTableType(options: Map[String, String]): String = {
|
||||
val params = mappingSqlOptionToHoodieParam(options)
|
||||
params.getOrElse(DataSourceWriteOptions.TABLE_TYPE.key,
|
||||
DataSourceWriteOptions.TABLE_TYPE.defaultValue)
|
||||
}
|
||||
|
||||
def getPreCombineField(options: Map[String, String]): Option[String] = {
|
||||
val params = mappingSqlOptionToHoodieParam(options)
|
||||
params.get(DataSourceWriteOptions.PRECOMBINE_FIELD.key).filter(_.nonEmpty)
|
||||
}
|
||||
|
||||
def deleteHoodieOptions(options: Map[String, String]): Map[String, String] = {
|
||||
options.filterNot(_._1.startsWith("hoodie.")).filterNot(kv => keyMapping.contains(kv._1))
|
||||
}
|
||||
|
||||
// extract primaryKey, preCombineField, type options
|
||||
def extractSqlOptions(options: Map[String, String]): Map[String, String] = {
|
||||
val sqlOptions = mappingTableConfigToSqlOption(options)
|
||||
val targetOptions = keyMapping.keySet -- Set(SQL_PAYLOAD_CLASS.sqlKeyName)
|
||||
sqlOptions.filterKeys(targetOptions.contains)
|
||||
}
|
||||
|
||||
// validate primaryKey, preCombineField and type options
|
||||
def validateTable(spark: SparkSession, schema: StructType, sqlOptions: Map[String, String]): Unit = {
|
||||
val resolver = spark.sessionState.conf.resolver
|
||||
|
||||
// validate primary key
|
||||
val primaryKeys = sqlOptions.get(SQL_KEY_TABLE_PRIMARY_KEY.sqlKeyName)
|
||||
.map(_.split(",").filter(_.length > 0))
|
||||
ValidationUtils.checkArgument(primaryKeys.nonEmpty, "No `primaryKey` is specified.")
|
||||
primaryKeys.get.foreach { primaryKey =>
|
||||
ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, primaryKey)),
|
||||
s"Can't find primaryKey `$primaryKey` in ${schema.treeString}.")
|
||||
}
|
||||
|
||||
// validate precombine key
|
||||
val precombineKey = sqlOptions.get(SQL_KEY_PRECOMBINE_FIELD.sqlKeyName)
|
||||
if (precombineKey.isDefined && precombineKey.get.nonEmpty) {
|
||||
ValidationUtils.checkArgument(schema.exists(f => resolver(f.name, precombineKey.get)),
|
||||
s"Can't find preCombineKey `${precombineKey.get}` in ${schema.treeString}.")
|
||||
}
|
||||
|
||||
// validate table type
|
||||
val tableType = sqlOptions.get(SQL_KEY_TABLE_TYPE.sqlKeyName)
|
||||
ValidationUtils.checkArgument(tableType.nonEmpty, "No `type` is specified.")
|
||||
ValidationUtils.checkArgument(
|
||||
tableType.get.equalsIgnoreCase(SQL_VALUE_TABLE_TYPE_COW) ||
|
||||
tableType.get.equalsIgnoreCase(SQL_VALUE_TABLE_TYPE_MOR),
|
||||
s"'type' must be '$SQL_VALUE_TABLE_TYPE_COW' or '$SQL_VALUE_TABLE_TYPE_MOR'")
|
||||
}
|
||||
|
||||
def buildConf[T](): HoodieSQLOptionBuilder[T] = {
|
||||
new HoodieSQLOptionBuilder[T]
|
||||
}
|
||||
}
|
||||
|
||||
case class HoodieSQLOption[T](
|
||||
sqlKeyName: String,
|
||||
hoodieKeyName: String,
|
||||
tableConfigKey: Option[String],
|
||||
defaultValue: Option[T]
|
||||
)
|
||||
|
||||
class HoodieSQLOptionBuilder[T] {
|
||||
|
||||
private var sqlKeyName: String = _
|
||||
private var hoodieKeyName: String =_
|
||||
private var tableConfigKey: String =_
|
||||
private var defaultValue: T =_
|
||||
|
||||
def withSqlKey(sqlKeyName: String): HoodieSQLOptionBuilder[T] = {
|
||||
this.sqlKeyName = sqlKeyName
|
||||
this
|
||||
}
|
||||
|
||||
def withHoodieKey(hoodieKeyName: String): HoodieSQLOptionBuilder[T] = {
|
||||
this.hoodieKeyName = hoodieKeyName
|
||||
this
|
||||
}
|
||||
|
||||
def withTableConfigKey(tableConfigKey: String): HoodieSQLOptionBuilder[T] = {
|
||||
this.tableConfigKey = tableConfigKey
|
||||
this
|
||||
}
|
||||
|
||||
def defaultValue(defaultValue: T): HoodieSQLOptionBuilder[T] = {
|
||||
this.defaultValue = defaultValue
|
||||
this
|
||||
}
|
||||
|
||||
def build(): HoodieSQLOption[T] = {
|
||||
HoodieSQLOption(sqlKeyName, hoodieKeyName, Option(tableConfigKey), Option(defaultValue))
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,304 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieMetadataConfig}
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstantTimeGenerator}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.{AvroConversionUtils, SparkAdapterSupport}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
|
||||
import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Expression}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
|
||||
import org.apache.spark.sql.execution.datasources.LogicalRelation
|
||||
import org.apache.spark.sql.internal.StaticSQLConf
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{Column, DataFrame, SparkSession}
|
||||
|
||||
import java.net.URI
|
||||
import java.text.SimpleDateFormat
|
||||
import java.util.{Locale, Properties}
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.immutable.Map
|
||||
|
||||
object HoodieSqlCommonUtils extends SparkAdapterSupport {
|
||||
// NOTE: {@code SimpleDataFormat} is NOT thread-safe
|
||||
// TODO replace w/ DateTimeFormatter
|
||||
private val defaultDateFormat =
|
||||
ThreadLocal.withInitial(new java.util.function.Supplier[SimpleDateFormat] {
|
||||
override def get() = new SimpleDateFormat("yyyy-MM-dd")
|
||||
})
|
||||
|
||||
def isHoodieTable(table: CatalogTable): Boolean = {
|
||||
table.provider.map(_.toLowerCase(Locale.ROOT)).orNull == "hudi"
|
||||
}
|
||||
|
||||
def isHoodieTable(tableId: TableIdentifier, spark: SparkSession): Boolean = {
|
||||
val table = spark.sessionState.catalog.getTableMetadata(tableId)
|
||||
isHoodieTable(table)
|
||||
}
|
||||
|
||||
def isHoodieTable(table: LogicalPlan, spark: SparkSession): Boolean = {
|
||||
tripAlias(table) match {
|
||||
case LogicalRelation(_, _, Some(tbl), _) => isHoodieTable(tbl)
|
||||
case relation: UnresolvedRelation =>
|
||||
isHoodieTable(sparkAdapter.toTableIdentifier(relation), spark)
|
||||
case _=> false
|
||||
}
|
||||
}
|
||||
|
||||
def getTableIdentifier(table: LogicalPlan): TableIdentifier = {
|
||||
table match {
|
||||
case SubqueryAlias(name, _) => sparkAdapter.toTableIdentifier(name)
|
||||
case _ => throw new IllegalArgumentException(s"Illegal table: $table")
|
||||
}
|
||||
}
|
||||
|
||||
def getTableSqlSchema(metaClient: HoodieTableMetaClient,
|
||||
includeMetadataFields: Boolean = false): Option[StructType] = {
|
||||
val schemaResolver = new TableSchemaResolver(metaClient)
|
||||
val avroSchema = try Some(schemaResolver.getTableAvroSchema(includeMetadataFields))
|
||||
catch {
|
||||
case _: Throwable => None
|
||||
}
|
||||
avroSchema.map(AvroConversionUtils.convertAvroSchemaToStructType)
|
||||
}
|
||||
|
||||
def getAllPartitionPaths(spark: SparkSession, table: CatalogTable): Seq[String] = {
|
||||
val sparkEngine = new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext))
|
||||
val metadataConfig = {
|
||||
val properties = new Properties()
|
||||
properties.putAll((spark.sessionState.conf.getAllConfs ++ table.storage.properties ++ table.properties).asJava)
|
||||
HoodieMetadataConfig.newBuilder.fromProperties(properties).build()
|
||||
}
|
||||
FSUtils.getAllPartitionPaths(sparkEngine, metadataConfig, getTableLocation(table, spark)).asScala
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is used to compatible with the old non-hive-styled partition table.
|
||||
* By default we enable the "hoodie.datasource.write.hive_style_partitioning"
|
||||
* when writing data to hudi table by spark sql by default.
|
||||
* If the exist table is a non-hive-styled partitioned table, we should
|
||||
* disable the "hoodie.datasource.write.hive_style_partitioning" when
|
||||
* merge or update the table. Or else, we will get an incorrect merge result
|
||||
* as the partition path mismatch.
|
||||
*/
|
||||
def isHiveStyledPartitioning(partitionPaths: Seq[String], table: CatalogTable): Boolean = {
|
||||
if (table.partitionColumnNames.nonEmpty) {
|
||||
val isHiveStylePartitionPath = (path: String) => {
|
||||
val fragments = path.split("/")
|
||||
if (fragments.size != table.partitionColumnNames.size) {
|
||||
false
|
||||
} else {
|
||||
fragments.zip(table.partitionColumnNames).forall {
|
||||
case (pathFragment, partitionColumn) => pathFragment.startsWith(s"$partitionColumn=")
|
||||
}
|
||||
}
|
||||
}
|
||||
partitionPaths.forall(isHiveStylePartitionPath)
|
||||
} else {
|
||||
true
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine whether URL encoding is enabled
|
||||
*/
|
||||
def isUrlEncodeEnabled(partitionPaths: Seq[String], table: CatalogTable): Boolean = {
|
||||
if (table.partitionColumnNames.nonEmpty) {
|
||||
partitionPaths.forall(partitionPath => partitionPath.split("/").length == table.partitionColumnNames.size)
|
||||
} else {
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
private def tripAlias(plan: LogicalPlan): LogicalPlan = {
|
||||
plan match {
|
||||
case SubqueryAlias(_, relation: LogicalPlan) =>
|
||||
tripAlias(relation)
|
||||
case other =>
|
||||
other
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add the hoodie meta fields to the schema.
|
||||
* @param schema
|
||||
* @return
|
||||
*/
|
||||
def addMetaFields(schema: StructType): StructType = {
|
||||
val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala
|
||||
// filter the meta field to avoid duplicate field.
|
||||
val dataFields = schema.fields.filterNot(f => metaFields.contains(f.name))
|
||||
val fields = metaFields.map(StructField(_, StringType)) ++ dataFields
|
||||
StructType(fields)
|
||||
}
|
||||
|
||||
private lazy val metaFields = HoodieRecord.HOODIE_META_COLUMNS.asScala.toSet
|
||||
|
||||
/**
|
||||
* Remove the meta fields from the schema.
|
||||
* @param schema
|
||||
* @return
|
||||
*/
|
||||
def removeMetaFields(schema: StructType): StructType = {
|
||||
StructType(schema.fields.filterNot(f => isMetaField(f.name)))
|
||||
}
|
||||
|
||||
def isMetaField(name: String): Boolean = {
|
||||
metaFields.contains(name)
|
||||
}
|
||||
|
||||
def removeMetaFields(df: DataFrame): DataFrame = {
|
||||
val withoutMetaColumns = df.logicalPlan.output
|
||||
.filterNot(attr => isMetaField(attr.name))
|
||||
.map(new Column(_))
|
||||
if (withoutMetaColumns.length != df.logicalPlan.output.size) {
|
||||
df.select(withoutMetaColumns: _*)
|
||||
} else {
|
||||
df
|
||||
}
|
||||
}
|
||||
|
||||
def removeMetaFields(attrs: Seq[Attribute]): Seq[Attribute] = {
|
||||
attrs.filterNot(attr => isMetaField(attr.name))
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the table location.
|
||||
* @param tableId
|
||||
* @param spark
|
||||
* @return
|
||||
*/
|
||||
def getTableLocation(tableId: TableIdentifier, spark: SparkSession): String = {
|
||||
val table = spark.sessionState.catalog.getTableMetadata(tableId)
|
||||
getTableLocation(table, spark)
|
||||
}
|
||||
|
||||
def getTableLocation(table: CatalogTable, sparkSession: SparkSession): String = {
|
||||
val uri = if (table.tableType == CatalogTableType.MANAGED && isHoodieTable(table)) {
|
||||
Some(sparkSession.sessionState.catalog.defaultTablePath(table.identifier))
|
||||
} else {
|
||||
table.storage.locationUri
|
||||
}
|
||||
val conf = sparkSession.sessionState.newHadoopConf()
|
||||
uri.map(makePathQualified(_, conf))
|
||||
.map(removePlaceHolder)
|
||||
.getOrElse(throw new IllegalArgumentException(s"Missing location for ${table.identifier}"))
|
||||
}
|
||||
|
||||
private def removePlaceHolder(path: String): String = {
|
||||
if (path == null || path.length == 0) {
|
||||
path
|
||||
} else if (path.endsWith("-__PLACEHOLDER__")) {
|
||||
path.substring(0, path.length() - 16)
|
||||
} else {
|
||||
path
|
||||
}
|
||||
}
|
||||
|
||||
def makePathQualified(path: URI, hadoopConf: Configuration): String = {
|
||||
val hadoopPath = new Path(path)
|
||||
val fs = hadoopPath.getFileSystem(hadoopConf)
|
||||
fs.makeQualified(hadoopPath).toUri.toString
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the hoodie.properties exists in the table path.
|
||||
*/
|
||||
def tableExistsInPath(tablePath: String, conf: Configuration): Boolean = {
|
||||
val basePath = new Path(tablePath)
|
||||
val fs = basePath.getFileSystem(conf)
|
||||
val metaPath = new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)
|
||||
fs.exists(metaPath)
|
||||
}
|
||||
|
||||
/**
|
||||
* Split the expression to a sub expression seq by the AND operation.
|
||||
* @param expression
|
||||
* @return
|
||||
*/
|
||||
def splitByAnd(expression: Expression): Seq[Expression] = {
|
||||
expression match {
|
||||
case And(left, right) =>
|
||||
splitByAnd(left) ++ splitByAnd(right)
|
||||
case exp => Seq(exp)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Append the spark config and table options to the baseConfig.
|
||||
*/
|
||||
def withSparkConf(spark: SparkSession, options: Map[String, String])
|
||||
(baseConfig: Map[String, String] = Map.empty): Map[String, String] = {
|
||||
baseConfig ++ DFSPropertiesConfiguration.getGlobalProps.asScala ++ // Table options has the highest priority
|
||||
(spark.sessionState.conf.getAllConfs ++ HoodieOptionConfig.mappingSqlOptionToHoodieParam(options))
|
||||
.filterKeys(_.startsWith("hoodie."))
|
||||
}
|
||||
|
||||
def isEnableHive(sparkSession: SparkSession): Boolean =
|
||||
"hive" == sparkSession.sessionState.conf.getConf(StaticSQLConf.CATALOG_IMPLEMENTATION)
|
||||
|
||||
/**
|
||||
* Convert different query instant time format to the commit time format.
|
||||
* Currently we support three kinds of instant time format for time travel query:
|
||||
* 1、yyyy-MM-dd HH:mm:ss
|
||||
* 2、yyyy-MM-dd
|
||||
* This will convert to 'yyyyMMdd000000'.
|
||||
* 3、yyyyMMddHHmmss
|
||||
*/
|
||||
def formatQueryInstant(queryInstant: String): String = {
|
||||
val instantLength = queryInstant.length
|
||||
if (instantLength == 19 || instantLength == 23) { // for yyyy-MM-dd HH:mm:ss[.SSS]
|
||||
HoodieInstantTimeGenerator.getInstantForDateString(queryInstant)
|
||||
} else if (instantLength == HoodieInstantTimeGenerator.SECS_INSTANT_ID_LENGTH
|
||||
|| instantLength == HoodieInstantTimeGenerator.MILLIS_INSTANT_ID_LENGTH) { // for yyyyMMddHHmmss[SSS]
|
||||
HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) // validate the format
|
||||
queryInstant
|
||||
} else if (instantLength == 10) { // for yyyy-MM-dd
|
||||
HoodieActiveTimeline.formatDate(defaultDateFormat.get().parse(queryInstant))
|
||||
} else {
|
||||
throw new IllegalArgumentException(s"Unsupported query instant time format: $queryInstant,"
|
||||
+ s"Supported time format are: 'yyyy-MM-dd: HH:mm:ss.SSS' or 'yyyy-MM-dd' or 'yyyyMMddHHmmssSSS'")
|
||||
}
|
||||
}
|
||||
|
||||
def formatName(sparkSession: SparkSession, name: String): String = {
|
||||
if (sparkSession.sessionState.conf.caseSensitiveAnalysis) name else name.toLowerCase(Locale.ROOT)
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if this is a empty table path.
|
||||
*/
|
||||
def isEmptyPath(tablePath: String, conf: Configuration): Boolean = {
|
||||
val basePath = new Path(tablePath)
|
||||
val fs = basePath.getFileSystem(conf)
|
||||
if (fs.exists(basePath)) {
|
||||
fs.listStatus(basePath).isEmpty
|
||||
} else {
|
||||
true
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
import java.io.ByteArrayOutputStream
|
||||
|
||||
import com.esotericsoftware.kryo.Kryo
|
||||
import com.esotericsoftware.kryo.io.{Input, Output}
|
||||
import org.apache.spark.SparkConf
|
||||
import org.apache.spark.serializer.KryoSerializer
|
||||
|
||||
|
||||
object SerDeUtils {
|
||||
|
||||
private val kryoLocal = new ThreadLocal[Kryo] {
|
||||
|
||||
override protected def initialValue: Kryo = {
|
||||
val serializer = new KryoSerializer(new SparkConf(true))
|
||||
serializer.newKryo()
|
||||
}
|
||||
}
|
||||
|
||||
def toBytes(o: Any): Array[Byte] = {
|
||||
val outputStream = new ByteArrayOutputStream(4096 * 5)
|
||||
val output = new Output(outputStream)
|
||||
try {
|
||||
kryoLocal.get.writeClassAndObject(output, o)
|
||||
output.flush()
|
||||
} finally {
|
||||
output.clear()
|
||||
output.close()
|
||||
}
|
||||
outputStream.toByteArray
|
||||
}
|
||||
|
||||
def toObject(bytes: Array[Byte]): Any = {
|
||||
val input = new Input(bytes)
|
||||
kryoLocal.get.readClassAndObject(input)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import java.nio.charset.StandardCharsets
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType}
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State
|
||||
import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant}
|
||||
import org.apache.hudi.common.util.{CommitUtils, Option}
|
||||
import org.apache.hudi.table.HoodieSparkTable
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, HoodieWriterUtils}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable}
|
||||
import org.apache.spark.sql.types.{StructField, StructType}
|
||||
import org.apache.spark.sql.util.SchemaUtils
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* Command for add new columns to the hudi table.
|
||||
*/
|
||||
case class AlterHoodieTableAddColumnsCommand(
|
||||
tableId: TableIdentifier,
|
||||
colsToAdd: Seq[StructField])
|
||||
extends HoodieLeafRunnableCommand {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
if (colsToAdd.nonEmpty) {
|
||||
val resolver = sparkSession.sessionState.conf.resolver
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableId)
|
||||
val tableSchema = hoodieCatalogTable.tableSchema
|
||||
val existsColumns =
|
||||
colsToAdd.map(_.name).filter(col => tableSchema.fieldNames.exists(f => resolver(f, col)))
|
||||
|
||||
if (existsColumns.nonEmpty) {
|
||||
throw new AnalysisException(s"Columns: [${existsColumns.mkString(",")}] already exists in the table," +
|
||||
s" table columns is: [${hoodieCatalogTable.tableSchemaWithoutMetaFields.fieldNames.mkString(",")}]")
|
||||
}
|
||||
// Get the new schema
|
||||
val newSqlSchema = StructType(tableSchema.fields ++ colsToAdd)
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableId.table)
|
||||
val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newSqlSchema, structName, nameSpace)
|
||||
|
||||
// Commit with new schema to change the table schema
|
||||
AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, hoodieCatalogTable, sparkSession)
|
||||
|
||||
// Refresh the new schema to meta
|
||||
val newDataSchema = StructType(hoodieCatalogTable.dataSchema.fields ++ colsToAdd)
|
||||
refreshSchemaInMeta(sparkSession, hoodieCatalogTable.table, newDataSchema)
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
private def refreshSchemaInMeta(sparkSession: SparkSession, table: CatalogTable,
|
||||
newSqlDataSchema: StructType): Unit = {
|
||||
try {
|
||||
sparkSession.catalog.uncacheTable(tableId.quotedString)
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
log.warn(s"Exception when attempting to uncache table ${tableId.quotedString}", e)
|
||||
}
|
||||
sparkSession.catalog.refreshTable(table.identifier.unquotedString)
|
||||
|
||||
SchemaUtils.checkColumnNameDuplication(
|
||||
newSqlDataSchema.map(_.name),
|
||||
"in the table definition of " + table.identifier,
|
||||
conf.caseSensitiveAnalysis)
|
||||
|
||||
sparkSession.sessionState.catalog.alterTableDataSchema(tableId, newSqlDataSchema)
|
||||
}
|
||||
}
|
||||
|
||||
object AlterHoodieTableAddColumnsCommand {
|
||||
/**
|
||||
* Generate an empty commit with new schema to change the table's schema.
|
||||
* @param schema The new schema to commit.
|
||||
* @param hoodieCatalogTable The hoodie catalog table.
|
||||
* @param sparkSession The spark session.
|
||||
*/
|
||||
def commitWithSchema(schema: Schema, hoodieCatalogTable: HoodieCatalogTable,
|
||||
sparkSession: SparkSession): Unit = {
|
||||
|
||||
val jsc = new JavaSparkContext(sparkSession.sparkContext)
|
||||
val client = DataSourceUtils.createHoodieClient(
|
||||
jsc,
|
||||
schema.toString,
|
||||
hoodieCatalogTable.tableLocation,
|
||||
hoodieCatalogTable.tableName,
|
||||
HoodieWriterUtils.parametersWithWriteDefaults(hoodieCatalogTable.catalogProperties).asJava
|
||||
)
|
||||
|
||||
val commitActionType = CommitUtils.getCommitActionType(WriteOperationType.INSERT, hoodieCatalogTable.tableType)
|
||||
val instantTime = HoodieActiveTimeline.createNewInstantTime
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
|
||||
val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext)
|
||||
val timeLine = hoodieTable.getActiveTimeline
|
||||
val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime)
|
||||
val metadata = new HoodieCommitMetadata
|
||||
metadata.setOperationType(WriteOperationType.INSERT)
|
||||
timeLine.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString.getBytes(StandardCharsets.UTF_8)))
|
||||
|
||||
client.commit(instantTime, jsc.emptyRDD)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,95 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.AvroConversionUtils
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
|
||||
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.types.{StructField, StructType}
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* Command for alter hudi table's column type.
|
||||
*/
|
||||
case class AlterHoodieTableChangeColumnCommand(
|
||||
tableIdentifier: TableIdentifier,
|
||||
columnName: String,
|
||||
newColumn: StructField)
|
||||
extends HoodieLeafRunnableCommand {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier)
|
||||
|
||||
val resolver = sparkSession.sessionState.conf.resolver
|
||||
if (!resolver(columnName, newColumn.name)) {
|
||||
throw new AnalysisException(s"Can not support change column name for hudi table currently.")
|
||||
}
|
||||
// Get the new schema
|
||||
val newTableSchema = StructType(
|
||||
hoodieCatalogTable.tableSchema.fields.map { field =>
|
||||
if (resolver(field.name, columnName)) {
|
||||
newColumn
|
||||
} else {
|
||||
field
|
||||
}
|
||||
})
|
||||
val newDataSchema = StructType(
|
||||
hoodieCatalogTable.dataSchema.fields.map { field =>
|
||||
if (resolver(field.name, columnName)) {
|
||||
newColumn
|
||||
} else {
|
||||
field
|
||||
}
|
||||
})
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tableIdentifier.table)
|
||||
val newSchema = AvroConversionUtils.convertStructTypeToAvroSchema(newTableSchema, structName, nameSpace)
|
||||
|
||||
// Validate the compatibility between new schema and origin schema.
|
||||
validateSchema(newSchema, hoodieCatalogTable.metaClient)
|
||||
// Commit new schema to change the table schema
|
||||
AlterHoodieTableAddColumnsCommand.commitWithSchema(newSchema, hoodieCatalogTable, sparkSession)
|
||||
|
||||
try {
|
||||
sparkSession.catalog.uncacheTable(tableIdentifier.quotedString)
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
log.warn(s"Exception when attempting to uncache table ${tableIdentifier.quotedString}", e)
|
||||
}
|
||||
sparkSession.catalog.refreshTable(tableIdentifier.unquotedString)
|
||||
// Change the schema in the meta using new data schema.
|
||||
sparkSession.sessionState.catalog.alterTableDataSchema(tableIdentifier, newDataSchema)
|
||||
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
private def validateSchema(newSchema: Schema, metaClient: HoodieTableMetaClient): Unit = {
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
val tableSchema = HoodieAvroUtils.createHoodieWriteSchema(schemaUtil.getTableAvroSchemaWithoutMetadataFields)
|
||||
if (!TableSchemaResolver.isSchemaCompatible(tableSchema, newSchema)) {
|
||||
throw new HoodieException("Failed schema compatibility check for newSchema :" + newSchema +
|
||||
", origin table schema :" + tableSchema + ", base path :" + metaClient.getBasePath)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,170 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.Resolver
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.execution.command.DDLUtils
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils._
|
||||
import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession}
|
||||
|
||||
case class AlterHoodieTableDropPartitionCommand(
|
||||
tableIdentifier: TableIdentifier,
|
||||
specs: Seq[TablePartitionSpec],
|
||||
ifExists : Boolean,
|
||||
purge : Boolean,
|
||||
retainData : Boolean)
|
||||
extends HoodieLeafRunnableCommand {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
val fullTableName = s"${tableIdentifier.database}.${tableIdentifier.table}"
|
||||
logInfo(s"start execute alter table drop partition command for $fullTableName")
|
||||
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier)
|
||||
|
||||
if (!hoodieCatalogTable.isPartitionedTable) {
|
||||
throw new AnalysisException(s"$fullTableName is a non-partitioned table that is not allowed to drop partition")
|
||||
}
|
||||
|
||||
DDLUtils.verifyAlterTableType(
|
||||
sparkSession.sessionState.catalog, hoodieCatalogTable.table, isView = false)
|
||||
|
||||
val normalizedSpecs: Seq[Map[String, String]] = specs.map { spec =>
|
||||
normalizePartitionSpec(
|
||||
spec,
|
||||
hoodieCatalogTable.partitionFields,
|
||||
hoodieCatalogTable.tableName,
|
||||
sparkSession.sessionState.conf.resolver)
|
||||
}
|
||||
|
||||
val partitionsToDrop = getPartitionPathToDrop(hoodieCatalogTable, normalizedSpecs)
|
||||
val parameters = buildHoodieConfig(sparkSession, hoodieCatalogTable, partitionsToDrop)
|
||||
HoodieSparkSqlWriter.write(
|
||||
sparkSession.sqlContext,
|
||||
SaveMode.Append,
|
||||
parameters,
|
||||
sparkSession.emptyDataFrame)
|
||||
|
||||
|
||||
// Recursively delete partition directories
|
||||
if (purge) {
|
||||
val engineContext = new HoodieSparkEngineContext(sparkSession.sparkContext)
|
||||
val basePath = hoodieCatalogTable.tableLocation
|
||||
val fullPartitionPath = FSUtils.getPartitionPath(basePath, partitionsToDrop)
|
||||
logInfo("Clean partition up " + fullPartitionPath)
|
||||
val fs = FSUtils.getFs(basePath, sparkSession.sparkContext.hadoopConfiguration)
|
||||
FSUtils.deleteDir(engineContext, fs, fullPartitionPath, sparkSession.sparkContext.defaultParallelism)
|
||||
}
|
||||
|
||||
sparkSession.catalog.refreshTable(tableIdentifier.unquotedString)
|
||||
logInfo(s"Finish execute alter table drop partition command for $fullTableName")
|
||||
Seq.empty[Row]
|
||||
}
|
||||
|
||||
private def buildHoodieConfig(
|
||||
sparkSession: SparkSession,
|
||||
hoodieCatalogTable: HoodieCatalogTable,
|
||||
partitionsToDrop: String): Map[String, String] = {
|
||||
val partitionFields = hoodieCatalogTable.partitionFields.mkString(",")
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
withSparkConf(sparkSession, Map.empty) {
|
||||
Map(
|
||||
"path" -> hoodieCatalogTable.tableLocation,
|
||||
TBL_NAME.key -> hoodieCatalogTable.tableName,
|
||||
TABLE_TYPE.key -> hoodieCatalogTable.tableTypeName,
|
||||
OPERATION.key -> DataSourceWriteOptions.DELETE_PARTITION_OPERATION_OPT_VAL,
|
||||
PARTITIONS_TO_DELETE.key -> partitionsToDrop,
|
||||
RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
|
||||
PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
|
||||
PARTITIONPATH_FIELD.key -> partitionFields,
|
||||
HIVE_SYNC_ENABLED.key -> enableHive.toString,
|
||||
META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HIVE_USE_JDBC.key -> "false",
|
||||
HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
|
||||
HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table,
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_PARTITION_FIELDS.key -> partitionFields,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
def normalizePartitionSpec[T](
|
||||
partitionSpec: Map[String, T],
|
||||
partColNames: Seq[String],
|
||||
tblName: String,
|
||||
resolver: Resolver): Map[String, T] = {
|
||||
val normalizedPartSpec = partitionSpec.toSeq.map { case (key, value) =>
|
||||
val normalizedKey = partColNames.find(resolver(_, key)).getOrElse {
|
||||
throw new AnalysisException(s"$key is not a valid partition column in table $tblName.")
|
||||
}
|
||||
normalizedKey -> value
|
||||
}
|
||||
|
||||
if (normalizedPartSpec.size < partColNames.size) {
|
||||
throw new AnalysisException(
|
||||
"All partition columns need to be specified for Hoodie's dropping partition")
|
||||
}
|
||||
|
||||
val lowerPartColNames = partColNames.map(_.toLowerCase)
|
||||
if (lowerPartColNames.distinct.length != lowerPartColNames.length) {
|
||||
val duplicateColumns = lowerPartColNames.groupBy(identity).collect {
|
||||
case (x, ys) if ys.length > 1 => s"`$x`"
|
||||
}
|
||||
throw new AnalysisException(
|
||||
s"Found duplicate column(s) in the partition schema: ${duplicateColumns.mkString(", ")}")
|
||||
}
|
||||
|
||||
normalizedPartSpec.toMap
|
||||
}
|
||||
|
||||
def getPartitionPathToDrop(
|
||||
hoodieCatalogTable: HoodieCatalogTable,
|
||||
normalizedSpecs: Seq[Map[String, String]]): String = {
|
||||
val table = hoodieCatalogTable.table
|
||||
val allPartitionPaths = hoodieCatalogTable.getAllPartitionPaths
|
||||
val enableHiveStylePartitioning = isHiveStyledPartitioning(allPartitionPaths, table)
|
||||
val enableEncodeUrl = isUrlEncodeEnabled(allPartitionPaths, table)
|
||||
val partitionsToDrop = normalizedSpecs.map { spec =>
|
||||
hoodieCatalogTable.partitionFields.map { partitionColumn =>
|
||||
val encodedPartitionValue = if (enableEncodeUrl) {
|
||||
PartitionPathEncodeUtils.escapePathName(spec(partitionColumn))
|
||||
} else {
|
||||
spec(partitionColumn)
|
||||
}
|
||||
if (enableHiveStylePartitioning) {
|
||||
partitionColumn + "=" + encodedPartitionValue
|
||||
} else {
|
||||
encodedPartitionValue
|
||||
}
|
||||
}.mkString("/")
|
||||
}.mkString(",")
|
||||
partitionsToDrop
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.execution.command.AlterTableRenameCommand
|
||||
|
||||
/**
|
||||
* Command for alter hudi table's table name.
|
||||
*/
|
||||
class AlterHoodieTableRenameCommand(
|
||||
oldName: TableIdentifier,
|
||||
newName: TableIdentifier,
|
||||
isView: Boolean)
|
||||
extends AlterTableRenameCommand(oldName, newName, isView) {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
if (newName != oldName) {
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, oldName)
|
||||
|
||||
// Init table with new name.
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(hoodieCatalogTable.tableConfig.getProps)
|
||||
.setTableName(newName.table)
|
||||
.initTable(hadoopConf, hoodieCatalogTable.tableLocation)
|
||||
|
||||
// Call AlterTableRenameCommand#run to rename table in meta.
|
||||
super.run(sparkSession)
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,242 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.hudi.common.model.HoodieFileFormat
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
|
||||
import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, TableAlreadyExistsException}
|
||||
import org.apache.spark.sql.catalyst.catalog._
|
||||
import org.apache.spark.sql.hive.HiveClientUtils
|
||||
import org.apache.spark.sql.hive.HiveExternalCatalog._
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlCommonUtils}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isEnableHive
|
||||
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.{SPARK_VERSION, SparkConf}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* Command for create hoodie table.
|
||||
*/
|
||||
case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
|
||||
extends HoodieLeafRunnableCommand with SparkAdapterSupport {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
val tableIsExists = sparkSession.sessionState.catalog.tableExists(table.identifier)
|
||||
if (tableIsExists) {
|
||||
if (ignoreIfExists) {
|
||||
// scalastyle:off
|
||||
return Seq.empty[Row]
|
||||
// scalastyle:on
|
||||
} else {
|
||||
throw new IllegalArgumentException(s"Table ${table.identifier.unquotedString} already exists.")
|
||||
}
|
||||
}
|
||||
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, table)
|
||||
// check if there are conflict between table configs defined in hoodie table and properties defined in catalog.
|
||||
CreateHoodieTableCommand.validateTblProperties(hoodieCatalogTable)
|
||||
// init hoodie table
|
||||
hoodieCatalogTable.initHoodieTable()
|
||||
|
||||
try {
|
||||
// create catalog table for this hoodie table
|
||||
CreateHoodieTableCommand.createTableInCatalog(sparkSession, hoodieCatalogTable, ignoreIfExists)
|
||||
} catch {
|
||||
case NonFatal(e) =>
|
||||
logWarning(s"Failed to create catalog table in metastore: ${e.getMessage}")
|
||||
}
|
||||
Seq.empty[Row]
|
||||
}
|
||||
}
|
||||
|
||||
object CreateHoodieTableCommand {
|
||||
|
||||
def validateTblProperties(hoodieCatalogTable: HoodieCatalogTable): Unit = {
|
||||
if (hoodieCatalogTable.hoodieTableExists) {
|
||||
val originTableConfig = hoodieCatalogTable.tableConfig.getProps.asScala.toMap
|
||||
val tableOptions = hoodieCatalogTable.catalogProperties
|
||||
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
|
||||
}
|
||||
}
|
||||
|
||||
def createTableInCatalog(sparkSession: SparkSession,
|
||||
hoodieCatalogTable: HoodieCatalogTable, ignoreIfExists: Boolean): Unit = {
|
||||
val table = hoodieCatalogTable.table
|
||||
assert(table.tableType != CatalogTableType.VIEW)
|
||||
|
||||
val catalog = sparkSession.sessionState.catalog
|
||||
val path = hoodieCatalogTable.tableLocation
|
||||
val tableConfig = hoodieCatalogTable.tableConfig
|
||||
val properties = tableConfig.getProps.asScala.toMap
|
||||
|
||||
val tableType = tableConfig.getTableType.name()
|
||||
val inputFormat = tableType match {
|
||||
case DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL =>
|
||||
classOf[HoodieParquetInputFormat].getCanonicalName
|
||||
case DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL =>
|
||||
classOf[HoodieParquetRealtimeInputFormat].getCanonicalName
|
||||
case _=> throw new IllegalArgumentException(s"UnKnow table type:$tableType")
|
||||
}
|
||||
val outputFormat = HoodieInputFormatUtils.getOutputFormatClassName(HoodieFileFormat.PARQUET)
|
||||
val serdeFormat = HoodieInputFormatUtils.getSerDeClassName(HoodieFileFormat.PARQUET)
|
||||
|
||||
// only parameters irrelevant to hudi can be set to storage.properties
|
||||
val storageProperties = HoodieOptionConfig.deleteHoodieOptions(properties)
|
||||
val newStorage = new CatalogStorageFormat(
|
||||
Some(new Path(path).toUri),
|
||||
Some(inputFormat),
|
||||
Some(outputFormat),
|
||||
Some(serdeFormat),
|
||||
table.storage.compressed,
|
||||
storageProperties + ("path" -> path))
|
||||
|
||||
val tablName = HoodieSqlCommonUtils.formatName(sparkSession, table.identifier.table)
|
||||
val newDatabaseName = HoodieSqlCommonUtils.formatName(sparkSession, table.identifier.database
|
||||
.getOrElse(catalog.getCurrentDatabase))
|
||||
|
||||
val newTableIdentifier = table.identifier
|
||||
.copy(table = tablName, database = Some(newDatabaseName))
|
||||
|
||||
// append pk, preCombineKey, type to the properties of table
|
||||
val newTblProperties = hoodieCatalogTable.catalogProperties ++ HoodieOptionConfig.extractSqlOptions(properties)
|
||||
val newTable = table.copy(
|
||||
identifier = newTableIdentifier,
|
||||
schema = hoodieCatalogTable.tableSchema,
|
||||
storage = newStorage,
|
||||
createVersion = SPARK_VERSION,
|
||||
properties = newTblProperties
|
||||
)
|
||||
|
||||
// Create table in the catalog
|
||||
val enableHive = isEnableHive(sparkSession)
|
||||
if (enableHive) {
|
||||
createHiveDataSourceTable(sparkSession, newTable, ignoreIfExists)
|
||||
} else {
|
||||
catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create Hive table for hudi.
|
||||
* Firstly, do some check for the schema & table.
|
||||
* Secondly, append some table properties need for spark datasource table.
|
||||
* Thirdly, create hive table using the HiveClient.
|
||||
* @param table
|
||||
* @param sparkSession
|
||||
*/
|
||||
private def createHiveDataSourceTable(sparkSession: SparkSession, table: CatalogTable,
|
||||
ignoreIfExists: Boolean): Unit = {
|
||||
val dbName = table.identifier.database.get
|
||||
// check database
|
||||
val dbExists = sparkSession.sessionState.catalog.databaseExists(dbName)
|
||||
if (!dbExists) {
|
||||
throw new NoSuchDatabaseException(dbName)
|
||||
}
|
||||
// check table exists
|
||||
if (sparkSession.sessionState.catalog.tableExists(table.identifier)) {
|
||||
throw new TableAlreadyExistsException(dbName, table.identifier.table)
|
||||
}
|
||||
// append some table properties need for spark data source table.
|
||||
val dataSourceProps = tableMetaToTableProps(sparkSession.sparkContext.conf,
|
||||
table, table.schema)
|
||||
|
||||
val tableWithDataSourceProps = table.copy(properties = dataSourceProps ++ table.properties)
|
||||
val client = HiveClientUtils.newClientForMetadata(sparkSession.sparkContext.conf,
|
||||
sparkSession.sessionState.newHadoopConf())
|
||||
// create hive table.
|
||||
client.createTable(tableWithDataSourceProps, ignoreIfExists)
|
||||
}
|
||||
|
||||
// This code is forked from org.apache.spark.sql.hive.HiveExternalCatalog#tableMetaToTableProps
|
||||
private def tableMetaToTableProps(sparkConf: SparkConf, table: CatalogTable,
|
||||
schema: StructType): Map[String, String] = {
|
||||
val partitionColumns = table.partitionColumnNames
|
||||
val bucketSpec = table.bucketSpec
|
||||
|
||||
val properties = new mutable.HashMap[String, String]
|
||||
properties.put(DATASOURCE_PROVIDER, "hudi")
|
||||
properties.put(CREATED_SPARK_VERSION, table.createVersion)
|
||||
|
||||
// Serialized JSON schema string may be too long to be stored into a single metastore table
|
||||
// property. In this case, we split the JSON string and store each part as a separate table
|
||||
// property.
|
||||
val threshold = sparkConf.get(SCHEMA_STRING_LENGTH_THRESHOLD)
|
||||
val schemaJsonString = schema.json
|
||||
// Split the JSON string.
|
||||
val parts = schemaJsonString.grouped(threshold).toSeq
|
||||
properties.put(DATASOURCE_SCHEMA_PREFIX + "numParts", parts.size.toString)
|
||||
parts.zipWithIndex.foreach { case (part, index) =>
|
||||
properties.put(s"$DATASOURCE_SCHEMA_PART_PREFIX$index", part)
|
||||
}
|
||||
|
||||
if (partitionColumns.nonEmpty) {
|
||||
properties.put(DATASOURCE_SCHEMA_NUMPARTCOLS, partitionColumns.length.toString)
|
||||
partitionColumns.zipWithIndex.foreach { case (partCol, index) =>
|
||||
properties.put(s"$DATASOURCE_SCHEMA_PARTCOL_PREFIX$index", partCol)
|
||||
}
|
||||
}
|
||||
|
||||
if (bucketSpec.isDefined) {
|
||||
val BucketSpec(numBuckets, bucketColumnNames, sortColumnNames) = bucketSpec.get
|
||||
|
||||
properties.put(DATASOURCE_SCHEMA_NUMBUCKETS, numBuckets.toString)
|
||||
properties.put(DATASOURCE_SCHEMA_NUMBUCKETCOLS, bucketColumnNames.length.toString)
|
||||
bucketColumnNames.zipWithIndex.foreach { case (bucketCol, index) =>
|
||||
properties.put(s"$DATASOURCE_SCHEMA_BUCKETCOL_PREFIX$index", bucketCol)
|
||||
}
|
||||
|
||||
if (sortColumnNames.nonEmpty) {
|
||||
properties.put(DATASOURCE_SCHEMA_NUMSORTCOLS, sortColumnNames.length.toString)
|
||||
sortColumnNames.zipWithIndex.foreach { case (sortCol, index) =>
|
||||
properties.put(s"$DATASOURCE_SCHEMA_SORTCOL_PREFIX$index", sortCol)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
properties.toMap
|
||||
}
|
||||
|
||||
private def checkTableConfigEqual(
|
||||
originTableConfig: Map[String, String],
|
||||
newTableConfig: Map[String, String],
|
||||
configKey: String): Unit = {
|
||||
if (originTableConfig.contains(configKey) && newTableConfig.contains(configKey)) {
|
||||
assert(originTableConfig(configKey) == newTableConfig(configKey),
|
||||
s"Table config: $configKey in the create table is: ${newTableConfig(configKey)}, is not the same with the value in " +
|
||||
s"hoodie.properties, which is: ${originTableConfig(configKey)}. Please keep the same.")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.catalyst.trees.HoodieLeafLike
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
|
||||
/**
|
||||
* Similar to `LeafRunnableCommand` in Spark3.2, `HoodieLeafRunnableCommand` mixed in
|
||||
* `HoodieLeafLike` can avoid subclasses of `RunnableCommand` to override
|
||||
* the `withNewChildrenInternal` method repeatedly.
|
||||
*/
|
||||
trait HoodieLeafRunnableCommand extends RunnableCommand with HoodieLeafLike[LogicalPlan]
|
||||
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils
|
||||
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
|
||||
import org.apache.spark.sql.execution.datasources.PartitioningUtils
|
||||
import org.apache.spark.sql.types.StringType
|
||||
|
||||
/**
|
||||
* Command for show hudi table's partitions.
|
||||
*/
|
||||
case class ShowHoodieTablePartitionsCommand(
|
||||
tableIdentifier: TableIdentifier,
|
||||
specOpt: Option[TablePartitionSpec])
|
||||
extends HoodieLeafRunnableCommand {
|
||||
|
||||
override val output: Seq[Attribute] = {
|
||||
AttributeReference("partition", StringType, nullable = false)() :: Nil
|
||||
}
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier)
|
||||
|
||||
val schemaOpt = hoodieCatalogTable.tableSchema
|
||||
val partitionColumnNamesOpt = hoodieCatalogTable.tableConfig.getPartitionFields
|
||||
|
||||
if (partitionColumnNamesOpt.isPresent && partitionColumnNamesOpt.get.nonEmpty && schemaOpt.nonEmpty) {
|
||||
if (specOpt.isEmpty) {
|
||||
hoodieCatalogTable.getAllPartitionPaths.map(Row(_))
|
||||
} else {
|
||||
val spec = specOpt.get
|
||||
hoodieCatalogTable.getAllPartitionPaths.filter { partitionPath =>
|
||||
val part = PartitioningUtils.parsePathFragment(partitionPath)
|
||||
spec.forall { case (col, value) =>
|
||||
PartitionPathEncodeUtils.escapePartitionValue(value) == part.getOrElse(col, null)
|
||||
}
|
||||
}.map(Row(_))
|
||||
}
|
||||
} else {
|
||||
Seq.empty[Row]
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,141 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.keygen._
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.spark.sql.Row
|
||||
import org.apache.spark.sql.types.{StructType, TimestampType}
|
||||
import org.joda.time.format.DateTimeFormat
|
||||
|
||||
import java.sql.Timestamp
|
||||
import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS}
|
||||
|
||||
/**
|
||||
* A complex key generator for sql command which do some process for the
|
||||
* timestamp data type partition field.
|
||||
*/
|
||||
class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props) {
|
||||
|
||||
private lazy val partitionSchema = {
|
||||
val partitionSchema = props.getString(SqlKeyGenerator.PARTITION_SCHEMA, "")
|
||||
if (partitionSchema != null && partitionSchema.nonEmpty) {
|
||||
Some(StructType.fromDDL(partitionSchema))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
// The origin key generator class for this table.
|
||||
private lazy val originKeyGen = {
|
||||
val beforeKeyGenClassName = props.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null)
|
||||
if (beforeKeyGenClassName != null) {
|
||||
val keyGenProps = new TypedProperties()
|
||||
keyGenProps.putAll(props)
|
||||
keyGenProps.remove(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
|
||||
val convertedKeyGenClassName = SqlKeyGenerator.getRealKeyGenClassName(props)
|
||||
keyGenProps.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, convertedKeyGenClassName)
|
||||
Some(KeyGenUtils.createKeyGeneratorByClassName(keyGenProps))
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
override def getRecordKey(record: GenericRecord): String = {
|
||||
if (originKeyGen.isDefined) {
|
||||
originKeyGen.get.getKey(record).getRecordKey
|
||||
} else {
|
||||
super.getRecordKey(record)
|
||||
}
|
||||
}
|
||||
|
||||
override def getRecordKey(row: Row): String = {
|
||||
if (originKeyGen.isDefined) {
|
||||
originKeyGen.get.asInstanceOf[SparkKeyGeneratorInterface].getRecordKey(row)
|
||||
} else {
|
||||
super.getRecordKey(row)
|
||||
}
|
||||
}
|
||||
|
||||
private def convertPartitionPathToSqlType(partitionPath: String, rowType: Boolean): String = {
|
||||
if (partitionSchema.isDefined) {
|
||||
// we can split the partitionPath here because we enable the URL_ENCODE_PARTITIONING_OPT
|
||||
// by default for sql.
|
||||
val partitionFragments = partitionPath.split(KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR)
|
||||
// If it is a table which is not write by spark sql before and the url encode has disabled,
|
||||
// the partition path level may not equal to the partition schema size. Just return the partitionPath
|
||||
// in this case.
|
||||
if (partitionFragments.size != partitionSchema.get.size) {
|
||||
partitionPath
|
||||
} else {
|
||||
partitionFragments.zip(partitionSchema.get.fields).map {
|
||||
case (partitionValue, partitionField) =>
|
||||
val hiveStylePrefix = s"${partitionField.name}="
|
||||
val isHiveStyle = partitionValue.startsWith(hiveStylePrefix)
|
||||
val _partitionValue = if (isHiveStyle) partitionValue.substring(hiveStylePrefix.length) else partitionValue
|
||||
|
||||
partitionField.dataType match {
|
||||
case TimestampType =>
|
||||
val timeMs = if (rowType) { // In RowType, the partitionPathValue is the time format string, convert to millis
|
||||
SqlKeyGenerator.sqlTimestampFormat.parseMillis(_partitionValue)
|
||||
} else {
|
||||
if (isConsistentLogicalTimestampEnabled) {
|
||||
Timestamp.valueOf(_partitionValue).getTime
|
||||
} else {
|
||||
MILLISECONDS.convert(_partitionValue.toLong, MICROSECONDS)
|
||||
}
|
||||
}
|
||||
val timestampFormat = PartitionPathEncodeUtils.escapePathName(
|
||||
SqlKeyGenerator.timestampTimeFormat.print(timeMs))
|
||||
if (isHiveStyle) s"$hiveStylePrefix$timestampFormat" else timestampFormat
|
||||
case _ => partitionValue
|
||||
}
|
||||
}.mkString(KeyGenUtils.DEFAULT_PARTITION_PATH_SEPARATOR)
|
||||
}
|
||||
} else partitionPath
|
||||
}
|
||||
|
||||
override def getPartitionPath(record: GenericRecord) = {
|
||||
val partitionPath = super.getPartitionPath(record)
|
||||
convertPartitionPathToSqlType(partitionPath, false)
|
||||
}
|
||||
|
||||
override def getPartitionPath(row: Row): String = {
|
||||
val partitionPath = super.getPartitionPath(row)
|
||||
convertPartitionPathToSqlType(partitionPath, true)
|
||||
}
|
||||
}
|
||||
|
||||
object SqlKeyGenerator {
|
||||
val PARTITION_SCHEMA = "hoodie.sql.partition.schema"
|
||||
val ORIGIN_KEYGEN_CLASS_NAME = "hoodie.sql.origin.keygen.class"
|
||||
private val timestampTimeFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
|
||||
private val sqlTimestampFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.S")
|
||||
|
||||
def getRealKeyGenClassName(props: TypedProperties): String = {
|
||||
val beforeKeyGenClassName = props.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null)
|
||||
if (beforeKeyGenClassName != null) {
|
||||
HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(beforeKeyGenClassName)
|
||||
} else {
|
||||
classOf[ComplexKeyGenerator].getCanonicalName
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
|
||||
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable
|
||||
import org.apache.spark.sql.execution.command.TruncateTableCommand
|
||||
|
||||
import scala.util.control.NonFatal
|
||||
|
||||
/**
|
||||
* Command for truncate hudi table.
|
||||
*/
|
||||
class TruncateHoodieTableCommand(
|
||||
tableIdentifier: TableIdentifier,
|
||||
partitionSpec: Option[TablePartitionSpec])
|
||||
extends TruncateTableCommand(tableIdentifier, partitionSpec) {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
val hoodieCatalogTable = HoodieCatalogTable(sparkSession, tableIdentifier)
|
||||
val properties = hoodieCatalogTable.tableConfig.getProps
|
||||
|
||||
try {
|
||||
// Delete all data in the table directory
|
||||
super.run(sparkSession)
|
||||
} catch {
|
||||
// TruncateTableCommand will delete the related directories first, and then refresh the table.
|
||||
// It will fail when refresh table, because the hudi meta directory(.hoodie) has been deleted at the first step.
|
||||
// So here ignore this failure, and refresh table later.
|
||||
case NonFatal(_) =>
|
||||
}
|
||||
|
||||
// If we have not specified the partition, truncate will delete all the data in the table path
|
||||
// include the hoodi.properties. In this case we should reInit the table.
|
||||
if (partitionSpec.isEmpty) {
|
||||
val hadoopConf = sparkSession.sessionState.newHadoopConf()
|
||||
// ReInit hoodie.properties
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(properties)
|
||||
.initTable(hadoopConf, hoodieCatalogTable.tableLocation)
|
||||
}
|
||||
|
||||
// After deleting the data, refresh the table to make sure we don't keep around a stale
|
||||
// file relation in the metastore cache and cached table data in the cache manager.
|
||||
sparkSession.catalog.refreshTable(hoodieCatalogTable.table.identifier.quotedString)
|
||||
Seq.empty[Row]
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* 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.command
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.{GenericRecord, IndexedRecord}
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord}
|
||||
import org.apache.hudi.common.util.{Option => HOption}
|
||||
import org.apache.hudi.exception.HoodieDuplicateKeyException
|
||||
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
/**
|
||||
* Validate the duplicate key for insert statement without enable the INSERT_DROP_DUPS_OPT
|
||||
* config.
|
||||
*/
|
||||
class ValidateDuplicateKeyPayload(record: GenericRecord, orderingVal: Comparable[_])
|
||||
extends DefaultHoodieRecordPayload(record, orderingVal) {
|
||||
|
||||
def this(record: HOption[GenericRecord]) {
|
||||
this(if (record.isPresent) record.get else null, 0)
|
||||
}
|
||||
|
||||
override def combineAndGetUpdateValue(currentValue: IndexedRecord,
|
||||
schema: Schema, properties: Properties): HOption[IndexedRecord] = {
|
||||
val key = currentValue.asInstanceOf[GenericRecord].get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString
|
||||
throw new HoodieDuplicateKeyException(key)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,191 @@
|
||||
/*
|
||||
* 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.command.payload
|
||||
|
||||
import java.util.UUID
|
||||
import org.apache.avro.generic.{GenericRecord, IndexedRecord}
|
||||
import org.apache.hudi.sql.IExpressionEvaluator
|
||||
import org.apache.spark.executor.InputMetrics
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.avro.AvroSerializer
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
|
||||
import org.apache.spark.sql.catalyst.expressions.codegen._
|
||||
import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, GenericInternalRow, LeafExpression, UnsafeArrayData, UnsafeMapData, UnsafeRow}
|
||||
import org.apache.spark.sql.catalyst.util.{ArrayData, MapData}
|
||||
import org.apache.spark.sql.hudi.command.payload.ExpressionCodeGen.RECORD_NAME
|
||||
import org.apache.spark.sql.types.{DataType, Decimal}
|
||||
import org.apache.spark.unsafe.Platform
|
||||
import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
|
||||
import org.apache.spark.util.ParentClassLoader
|
||||
import org.apache.spark.{TaskContext, TaskKilledException}
|
||||
import org.codehaus.commons.compiler.CompileException
|
||||
import org.codehaus.janino.{ClassBodyEvaluator, InternalCompilerException}
|
||||
|
||||
/**
|
||||
* Do CodeGen for expression based on IndexedRecord.
|
||||
* The mainly difference with the spark's CodeGen for expression is that
|
||||
* the expression's input is a IndexedRecord but not a Row.
|
||||
*
|
||||
*/
|
||||
object ExpressionCodeGen extends Logging {
|
||||
|
||||
val RECORD_NAME = "record"
|
||||
|
||||
/**
|
||||
* CodeGen for expressions.
|
||||
* @param exprs The expression list to CodeGen.
|
||||
* @return An IExpressionEvaluator generate by CodeGen which take a IndexedRecord as input
|
||||
* param and return a Array of results for each expression.
|
||||
*/
|
||||
def doCodeGen(exprs: Seq[Expression], serializer: AvroSerializer): IExpressionEvaluator = {
|
||||
val ctx = new CodegenContext()
|
||||
// Set the input_row to null as we do not use row as the input object but Record.
|
||||
ctx.INPUT_ROW = null
|
||||
|
||||
val replacedExprs = exprs.map(replaceBoundReference)
|
||||
val resultVars = replacedExprs.map(_.genCode(ctx))
|
||||
val className = s"ExpressionPayloadEvaluator_${UUID.randomUUID().toString.replace("-", "_")}"
|
||||
val codeBody =
|
||||
s"""
|
||||
|private Object[] references;
|
||||
|private String code;
|
||||
|private AvroSerializer serializer;
|
||||
|
|
||||
|public $className(Object references, String code, AvroSerializer serializer) {
|
||||
| this.references = (Object[])references;
|
||||
| this.code = code;
|
||||
| this.serializer = serializer;
|
||||
|}
|
||||
|
|
||||
|public GenericRecord eval(IndexedRecord $RECORD_NAME) {
|
||||
| ${resultVars.map(_.code).mkString("\n")}
|
||||
| Object[] results = new Object[${resultVars.length}];
|
||||
| ${
|
||||
(for (i <- resultVars.indices) yield {
|
||||
s"""
|
||||
|if (${resultVars(i).isNull}) {
|
||||
| results[$i] = null;
|
||||
|} else {
|
||||
| results[$i] = ${resultVars(i).value.code};
|
||||
|}
|
||||
""".stripMargin
|
||||
}).mkString("\n")
|
||||
}
|
||||
InternalRow row = new GenericInternalRow(results);
|
||||
return (GenericRecord) serializer.serialize(row);
|
||||
| }
|
||||
|
|
||||
|public String getCode() {
|
||||
| return code;
|
||||
|}
|
||||
""".stripMargin
|
||||
|
||||
val evaluator = new ClassBodyEvaluator()
|
||||
val parentClassLoader = new ParentClassLoader(
|
||||
Option(Thread.currentThread().getContextClassLoader).getOrElse(getClass.getClassLoader))
|
||||
|
||||
evaluator.setParentClassLoader(parentClassLoader)
|
||||
// Cannot be under package codegen, or fail with java.lang.InstantiationException
|
||||
evaluator.setClassName(s"org.apache.hudi.sql.payload.$className")
|
||||
evaluator.setDefaultImports(
|
||||
classOf[Platform].getName,
|
||||
classOf[InternalRow].getName,
|
||||
classOf[UnsafeRow].getName,
|
||||
classOf[UTF8String].getName,
|
||||
classOf[Decimal].getName,
|
||||
classOf[CalendarInterval].getName,
|
||||
classOf[ArrayData].getName,
|
||||
classOf[UnsafeArrayData].getName,
|
||||
classOf[MapData].getName,
|
||||
classOf[UnsafeMapData].getName,
|
||||
classOf[Expression].getName,
|
||||
classOf[TaskContext].getName,
|
||||
classOf[TaskKilledException].getName,
|
||||
classOf[InputMetrics].getName,
|
||||
classOf[IndexedRecord].getName,
|
||||
classOf[AvroSerializer].getName,
|
||||
classOf[GenericRecord].getName,
|
||||
classOf[GenericInternalRow].getName
|
||||
)
|
||||
evaluator.setImplementedInterfaces(Array(classOf[IExpressionEvaluator]))
|
||||
try {
|
||||
evaluator.cook(codeBody)
|
||||
} catch {
|
||||
case e: InternalCompilerException =>
|
||||
val msg = s"failed to compile: $e"
|
||||
logError(msg, e)
|
||||
throw new InternalCompilerException(msg, e)
|
||||
case e: CompileException =>
|
||||
val msg = s"failed to compile: $e"
|
||||
logError(msg, e)
|
||||
throw new CompileException(msg, e.getLocation)
|
||||
}
|
||||
val referenceArray = ctx.references.toArray.map(_.asInstanceOf[Object])
|
||||
val expressionSql = exprs.map(_.sql).mkString(" ")
|
||||
|
||||
evaluator.getClazz.getConstructor(classOf[Object], classOf[String], classOf[AvroSerializer])
|
||||
.newInstance(referenceArray, s"Expressions is: [$expressionSql]\nCodeBody is: {\n$codeBody\n}", serializer)
|
||||
.asInstanceOf[IExpressionEvaluator]
|
||||
}
|
||||
|
||||
/**
|
||||
* Replace the BoundReference to the Record implement which will override the
|
||||
* doGenCode method.
|
||||
*/
|
||||
private def replaceBoundReference(expression: Expression): Expression = {
|
||||
expression transformDown {
|
||||
case BoundReference(ordinal, dataType, nullable) =>
|
||||
RecordBoundReference(ordinal, dataType, nullable)
|
||||
case other =>
|
||||
other
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
case class RecordBoundReference(ordinal: Int, dataType: DataType, nullable: Boolean)
|
||||
extends LeafExpression {
|
||||
|
||||
/**
|
||||
* Do the CodeGen for RecordBoundReference.
|
||||
* Use "IndexedRecord" as the input object but not a "Row"
|
||||
*/
|
||||
override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
|
||||
val javaType = JavaCode.javaType(dataType)
|
||||
val boxType = JavaCode.boxedType(dataType)
|
||||
|
||||
val value = s"($boxType)$RECORD_NAME.get($ordinal)"
|
||||
if (nullable) {
|
||||
ev.copy(code =
|
||||
code"""
|
||||
| boolean ${ev.isNull} = $RECORD_NAME.get($ordinal) == null;
|
||||
| $javaType ${ev.value} = ${ev.isNull} ?
|
||||
| ${CodeGenerator.defaultValue(dataType)} : ($value);
|
||||
"""
|
||||
)
|
||||
} else {
|
||||
ev.copy(code = code"$javaType ${ev.value} = $value;", isNull = FalseLiteral)
|
||||
}
|
||||
}
|
||||
|
||||
override def eval(input: InternalRow): Any = {
|
||||
throw new IllegalArgumentException(s"Should not call eval method for " +
|
||||
s"${getClass.getCanonicalName}")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,326 @@
|
||||
/*
|
||||
* 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.command.payload
|
||||
|
||||
import java.util.{Base64, Properties}
|
||||
import java.util.concurrent.Callable
|
||||
|
||||
import com.google.common.cache.CacheBuilder
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.{GenericData, GenericRecord, IndexedRecord}
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.avro.HoodieAvroUtils.bytesToAvro
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodiePayloadProps, HoodieRecord}
|
||||
import org.apache.hudi.common.util.{ValidationUtils, Option => HOption}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.io.HoodieWriteHandle
|
||||
import org.apache.hudi.sql.IExpressionEvaluator
|
||||
|
||||
import org.apache.spark.sql.avro.{AvroSerializer, HoodieAvroSerializer, SchemaConverters}
|
||||
import org.apache.spark.sql.catalyst.expressions.Expression
|
||||
import org.apache.spark.sql.hudi.SerDeUtils
|
||||
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload.getEvaluator
|
||||
import org.apache.spark.sql.types.{StructField, StructType}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable.ArrayBuffer
|
||||
|
||||
/**
|
||||
* A HoodieRecordPayload for MergeIntoHoodieTableCommand.
|
||||
* It will execute the condition and assignments expression in the
|
||||
* match and not-match actions and compute the final record to write.
|
||||
*
|
||||
* If there is no condition match the record, ExpressionPayload will return
|
||||
* a HoodieWriteHandle.IGNORE_RECORD, and the write handles will ignore this record.
|
||||
*/
|
||||
class ExpressionPayload(record: GenericRecord,
|
||||
orderingVal: Comparable[_])
|
||||
extends DefaultHoodieRecordPayload(record, orderingVal) {
|
||||
|
||||
def this(recordOpt: HOption[GenericRecord]) {
|
||||
this(recordOpt.orElse(null), 0)
|
||||
}
|
||||
|
||||
/**
|
||||
* The schema of this table.
|
||||
*/
|
||||
private var writeSchema: Schema = _
|
||||
|
||||
override def combineAndGetUpdateValue(currentValue: IndexedRecord,
|
||||
schema: Schema): HOption[IndexedRecord] = {
|
||||
throw new IllegalStateException(s"Should not call this method for ${getClass.getCanonicalName}")
|
||||
}
|
||||
|
||||
override def getInsertValue(schema: Schema): HOption[IndexedRecord] = {
|
||||
throw new IllegalStateException(s"Should not call this method for ${getClass.getCanonicalName}")
|
||||
}
|
||||
|
||||
override def combineAndGetUpdateValue(targetRecord: IndexedRecord,
|
||||
schema: Schema, properties: Properties): HOption[IndexedRecord] = {
|
||||
val sourceRecord = bytesToAvro(recordBytes, schema)
|
||||
val joinSqlRecord = new SqlTypedRecord(joinRecord(sourceRecord, targetRecord))
|
||||
processMatchedRecord(joinSqlRecord, Some(targetRecord), properties)
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the matched record. Firstly test if the record matched any of the update-conditions,
|
||||
* if matched, return the update assignments result. Secondly, test if the record matched
|
||||
* delete-condition, if matched then return a delete record. Finally if no condition matched,
|
||||
* return a {@link HoodieWriteHandle.IGNORE_RECORD} which will be ignored by HoodieWriteHandle.
|
||||
* @param inputRecord The input record to process.
|
||||
* @param targetRecord The origin exist record.
|
||||
* @param properties The properties.
|
||||
* @return The result of the record to update or delete.
|
||||
*/
|
||||
private def processMatchedRecord(inputRecord: SqlTypedRecord,
|
||||
targetRecord: Option[IndexedRecord], properties: Properties): HOption[IndexedRecord] = {
|
||||
// Process update
|
||||
val updateConditionAndAssignmentsText =
|
||||
properties.get(ExpressionPayload.PAYLOAD_UPDATE_CONDITION_AND_ASSIGNMENTS)
|
||||
assert(updateConditionAndAssignmentsText != null,
|
||||
s"${ExpressionPayload.PAYLOAD_UPDATE_CONDITION_AND_ASSIGNMENTS} have not set")
|
||||
|
||||
var resultRecordOpt: HOption[IndexedRecord] = null
|
||||
|
||||
// Get the Evaluator for each condition and update assignments.
|
||||
initWriteSchemaIfNeed(properties)
|
||||
val updateConditionAndAssignments = getEvaluator(updateConditionAndAssignmentsText.toString, writeSchema)
|
||||
for ((conditionEvaluator, assignmentEvaluator) <- updateConditionAndAssignments
|
||||
if resultRecordOpt == null) {
|
||||
val conditionVal = evaluate(conditionEvaluator, inputRecord).get(0).asInstanceOf[Boolean]
|
||||
// If the update condition matched then execute assignment expression
|
||||
// to compute final record to update. We will return the first matched record.
|
||||
if (conditionVal) {
|
||||
val resultRecord = evaluate(assignmentEvaluator, inputRecord)
|
||||
|
||||
if (targetRecord.isEmpty || needUpdatingPersistedRecord(targetRecord.get, resultRecord, properties)) {
|
||||
resultRecordOpt = HOption.of(resultRecord)
|
||||
} else {
|
||||
// if the PreCombine field value of targetRecord is greate
|
||||
// than the new incoming record, just keep the old record value.
|
||||
resultRecordOpt = HOption.of(targetRecord.get)
|
||||
}
|
||||
}
|
||||
}
|
||||
if (resultRecordOpt == null) {
|
||||
// Process delete
|
||||
val deleteConditionText = properties.get(ExpressionPayload.PAYLOAD_DELETE_CONDITION)
|
||||
if (deleteConditionText != null) {
|
||||
val deleteCondition = getEvaluator(deleteConditionText.toString, writeSchema).head._1
|
||||
val deleteConditionVal = evaluate(deleteCondition, inputRecord).get(0).asInstanceOf[Boolean]
|
||||
if (deleteConditionVal) {
|
||||
resultRecordOpt = HOption.empty()
|
||||
}
|
||||
}
|
||||
}
|
||||
if (resultRecordOpt == null) {
|
||||
// If there is no condition matched, just filter this record.
|
||||
// here we return a IGNORE_RECORD, HoodieMergeHandle will not handle it.
|
||||
HOption.of(HoodieWriteHandle.IGNORE_RECORD)
|
||||
} else {
|
||||
resultRecordOpt
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the not-matched record. Test if the record matched any of insert-conditions,
|
||||
* if matched then return the result of insert-assignment. Or else return a
|
||||
* {@link HoodieWriteHandle.IGNORE_RECORD} which will be ignored by HoodieWriteHandle.
|
||||
*
|
||||
* @param inputRecord The input record to process.
|
||||
* @param properties The properties.
|
||||
* @return The result of the record to insert.
|
||||
*/
|
||||
private def processNotMatchedRecord(inputRecord: SqlTypedRecord, properties: Properties): HOption[IndexedRecord] = {
|
||||
val insertConditionAndAssignmentsText =
|
||||
properties.get(ExpressionPayload.PAYLOAD_INSERT_CONDITION_AND_ASSIGNMENTS)
|
||||
// Get the evaluator for each condition and insert assignment.
|
||||
initWriteSchemaIfNeed(properties)
|
||||
val insertConditionAndAssignments =
|
||||
ExpressionPayload.getEvaluator(insertConditionAndAssignmentsText.toString, writeSchema)
|
||||
var resultRecordOpt: HOption[IndexedRecord] = null
|
||||
for ((conditionEvaluator, assignmentEvaluator) <- insertConditionAndAssignments
|
||||
if resultRecordOpt == null) {
|
||||
val conditionVal = evaluate(conditionEvaluator, inputRecord).get(0).asInstanceOf[Boolean]
|
||||
// If matched the insert condition then execute the assignment expressions to compute the
|
||||
// result record. We will return the first matched record.
|
||||
if (conditionVal) {
|
||||
val resultRecord = evaluate(assignmentEvaluator, inputRecord)
|
||||
resultRecordOpt = HOption.of(resultRecord)
|
||||
}
|
||||
}
|
||||
if (resultRecordOpt != null) {
|
||||
resultRecordOpt
|
||||
} else {
|
||||
// If there is no condition matched, just filter this record.
|
||||
// Here we return a IGNORE_RECORD, HoodieCreateHandle will not handle it.
|
||||
HOption.of(HoodieWriteHandle.IGNORE_RECORD)
|
||||
}
|
||||
}
|
||||
|
||||
override def getInsertValue(schema: Schema, properties: Properties): HOption[IndexedRecord] = {
|
||||
val incomingRecord = bytesToAvro(recordBytes, schema)
|
||||
if (isDeleteRecord(incomingRecord)) {
|
||||
HOption.empty[IndexedRecord]()
|
||||
} else {
|
||||
val sqlTypedRecord = new SqlTypedRecord(incomingRecord)
|
||||
if (isMORTable(properties)) {
|
||||
// For the MOR table, both the matched and not-matched record will step into the getInsertValue() method.
|
||||
// We call the processMatchedRecord() method if current is a Update-Record to process
|
||||
// the matched record. Or else we call processNotMatchedRecord() method to process the not matched record.
|
||||
val isUpdateRecord = properties.getProperty(HoodiePayloadProps.PAYLOAD_IS_UPDATE_RECORD_FOR_MOR, "false").toBoolean
|
||||
if (isUpdateRecord) {
|
||||
processMatchedRecord(sqlTypedRecord, Option.empty, properties)
|
||||
} else {
|
||||
processNotMatchedRecord(sqlTypedRecord, properties)
|
||||
}
|
||||
} else {
|
||||
// For COW table, only the not-matched record will step into the getInsertValue method, So just call
|
||||
// the processNotMatchedRecord() here.
|
||||
processNotMatchedRecord(sqlTypedRecord, properties)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private def isMORTable(properties: Properties): Boolean = {
|
||||
properties.getProperty(TABLE_TYPE.key, null) == MOR_TABLE_TYPE_OPT_VAL
|
||||
}
|
||||
|
||||
private def convertToRecord(values: Array[AnyRef], schema: Schema): IndexedRecord = {
|
||||
assert(values.length == schema.getFields.size())
|
||||
val writeRecord = new GenericData.Record(schema)
|
||||
for (i <- values.indices) {
|
||||
writeRecord.put(i, values(i))
|
||||
}
|
||||
writeRecord
|
||||
}
|
||||
|
||||
/**
|
||||
* Init the table schema.
|
||||
*/
|
||||
private def initWriteSchemaIfNeed(properties: Properties): Unit = {
|
||||
if (writeSchema == null) {
|
||||
ValidationUtils.checkArgument(properties.containsKey(HoodieWriteConfig.WRITE_SCHEMA.key),
|
||||
s"Missing ${HoodieWriteConfig.WRITE_SCHEMA.key}")
|
||||
writeSchema = new Schema.Parser().parse(properties.getProperty(HoodieWriteConfig.WRITE_SCHEMA.key))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Join the source record with the target record.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
private def joinRecord(sourceRecord: IndexedRecord, targetRecord: IndexedRecord): IndexedRecord = {
|
||||
val leftSchema = sourceRecord.getSchema
|
||||
// the targetRecord is load from the disk, it contains the meta fields, so we remove it here
|
||||
val rightSchema = HoodieAvroUtils.removeMetadataFields(targetRecord.getSchema)
|
||||
val joinSchema = mergeSchema(leftSchema, rightSchema)
|
||||
|
||||
val values = new ArrayBuffer[AnyRef]()
|
||||
for (i <- 0 until joinSchema.getFields.size()) {
|
||||
val value = if (i < leftSchema.getFields.size()) {
|
||||
sourceRecord.get(i)
|
||||
} else { // skip meta field
|
||||
targetRecord.get(i - leftSchema.getFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size())
|
||||
}
|
||||
values += value
|
||||
}
|
||||
convertToRecord(values.toArray, joinSchema)
|
||||
}
|
||||
|
||||
private def mergeSchema(a: Schema, b: Schema): Schema = {
|
||||
val mergedFields =
|
||||
a.getFields.asScala.map(field =>
|
||||
new Schema.Field("a_" + field.name,
|
||||
field.schema, field.doc, field.defaultVal, field.order)) ++
|
||||
b.getFields.asScala.map(field =>
|
||||
new Schema.Field("b_" + field.name,
|
||||
field.schema, field.doc, field.defaultVal, field.order))
|
||||
Schema.createRecord(a.getName, a.getDoc, a.getNamespace, a.isError, mergedFields.asJava)
|
||||
}
|
||||
|
||||
private def evaluate(evaluator: IExpressionEvaluator, sqlTypedRecord: SqlTypedRecord): GenericRecord = {
|
||||
try evaluator.eval(sqlTypedRecord) catch {
|
||||
case e: Throwable =>
|
||||
throw new RuntimeException(s"Error in execute expression: ${e.getMessage}.\n${evaluator.getCode}", e)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
object ExpressionPayload {
|
||||
|
||||
/**
|
||||
* Property for pass the merge-into delete clause condition expresssion.
|
||||
*/
|
||||
val PAYLOAD_DELETE_CONDITION = "hoodie.payload.delete.condition"
|
||||
|
||||
/**
|
||||
* Property for pass the merge-into update clauses's condition and assignments.
|
||||
*/
|
||||
val PAYLOAD_UPDATE_CONDITION_AND_ASSIGNMENTS = "hoodie.payload.update.condition.assignments"
|
||||
|
||||
/**
|
||||
* Property for pass the merge-into insert clauses's condition and assignments.
|
||||
*/
|
||||
val PAYLOAD_INSERT_CONDITION_AND_ASSIGNMENTS = "hoodie.payload.insert.condition.assignments"
|
||||
|
||||
/**
|
||||
* A cache for the serializedConditionAssignments to the compiled class after CodeGen.
|
||||
* The Map[IExpressionEvaluator, IExpressionEvaluator] is the map of the condition expression
|
||||
* to the assignments expression.
|
||||
*/
|
||||
private val cache = CacheBuilder.newBuilder()
|
||||
.maximumSize(1024)
|
||||
.build[String, Map[IExpressionEvaluator, IExpressionEvaluator]]()
|
||||
|
||||
/**
|
||||
* Do the CodeGen for each condition and assignment expressions.We will cache it to reduce
|
||||
* the compile time for each method call.
|
||||
*/
|
||||
def getEvaluator(
|
||||
serializedConditionAssignments: String, writeSchema: Schema): Map[IExpressionEvaluator, IExpressionEvaluator] = {
|
||||
cache.get(serializedConditionAssignments,
|
||||
new Callable[Map[IExpressionEvaluator, IExpressionEvaluator]] {
|
||||
|
||||
override def call(): Map[IExpressionEvaluator, IExpressionEvaluator] = {
|
||||
val serializedBytes = Base64.getDecoder.decode(serializedConditionAssignments)
|
||||
val conditionAssignments = SerDeUtils.toObject(serializedBytes)
|
||||
.asInstanceOf[Map[Expression, Seq[Expression]]]
|
||||
// Do the CodeGen for condition expression and assignment expression
|
||||
conditionAssignments.map {
|
||||
case (condition, assignments) =>
|
||||
val conditionType = StructType(Seq(StructField("_col0", condition.dataType, nullable = true)))
|
||||
val conditionSerializer = new AvroSerializer(conditionType,
|
||||
SchemaConverters.toAvroType(conditionType), false)
|
||||
val conditionEvaluator = ExpressionCodeGen.doCodeGen(Seq(condition), conditionSerializer)
|
||||
|
||||
val assignSqlType = AvroConversionUtils.convertAvroSchemaToStructType(writeSchema)
|
||||
val assignSerializer = new HoodieAvroSerializer(assignSqlType, writeSchema, false)
|
||||
val assignmentEvaluator = ExpressionCodeGen.doCodeGen(assignments, assignSerializer)
|
||||
conditionEvaluator -> assignmentEvaluator
|
||||
}
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* 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.command.payload
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord
|
||||
import org.apache.avro.Schema
|
||||
|
||||
import org.apache.hudi.AvroConversionUtils
|
||||
|
||||
import org.apache.spark.sql.avro.HoodieAvroDeserializer
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
|
||||
/**
|
||||
* A sql typed record which will convert the avro field to sql typed value.
|
||||
*/
|
||||
class SqlTypedRecord(val record: IndexedRecord) extends IndexedRecord {
|
||||
|
||||
private lazy val sqlType = AvroConversionUtils.convertAvroSchemaToStructType(getSchema)
|
||||
private lazy val avroDeserializer = HoodieAvroDeserializer(record.getSchema, sqlType)
|
||||
private lazy val sqlRow = avroDeserializer.deserializeData(record).asInstanceOf[InternalRow]
|
||||
|
||||
override def put(i: Int, v: Any): Unit = {
|
||||
record.put(i, v)
|
||||
}
|
||||
|
||||
override def get(i: Int): AnyRef = {
|
||||
sqlRow.get(i, sqlType(i).dataType)
|
||||
}
|
||||
|
||||
override def getSchema: Schema = record.getSchema
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.spark.sql.hudi.streaming
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonInclude.Include
|
||||
import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper}
|
||||
import com.fasterxml.jackson.module.scala.DefaultScalaModule
|
||||
import com.fasterxml.jackson.module.scala.experimental.ScalaObjectMapper
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline
|
||||
import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset}
|
||||
|
||||
case class HoodieSourceOffset(commitTime: String) extends Offset {
|
||||
|
||||
override def json(): String = {
|
||||
HoodieSourceOffset.toJson(this)
|
||||
}
|
||||
|
||||
override def equals(obj: Any): Boolean = {
|
||||
obj match {
|
||||
case HoodieSourceOffset(otherCommitTime) =>
|
||||
otherCommitTime == commitTime
|
||||
case _=> false
|
||||
}
|
||||
}
|
||||
|
||||
override def hashCode(): Int = {
|
||||
commitTime.hashCode
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
object HoodieSourceOffset {
|
||||
val mapper = new ObjectMapper with ScalaObjectMapper
|
||||
mapper.setSerializationInclusion(Include.NON_ABSENT)
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false)
|
||||
mapper.registerModule(DefaultScalaModule)
|
||||
|
||||
def toJson(offset: HoodieSourceOffset): String = {
|
||||
mapper.writeValueAsString(offset)
|
||||
}
|
||||
|
||||
def fromJson(json: String): HoodieSourceOffset = {
|
||||
mapper.readValue[HoodieSourceOffset](json)
|
||||
}
|
||||
|
||||
def apply(offset: Offset): HoodieSourceOffset = {
|
||||
offset match {
|
||||
case SerializedOffset(json) => fromJson(json)
|
||||
case o: HoodieSourceOffset => o
|
||||
}
|
||||
}
|
||||
|
||||
val INIT_OFFSET = HoodieSourceOffset(HoodieTimeline.INIT_INSTANT_TS)
|
||||
}
|
||||
@@ -0,0 +1,197 @@
|
||||
/*
|
||||
* 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.streaming
|
||||
|
||||
import java.io.{BufferedWriter, InputStream, OutputStream, OutputStreamWriter}
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util.Date
|
||||
|
||||
import org.apache.hadoop.fs.Path
|
||||
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, IncrementalRelation, MergeOnReadIncrementalRelation, SparkAdapterSupport}
|
||||
import org.apache.hudi.common.model.HoodieTableType
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{FileIOUtils, TablePathUtils}
|
||||
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieStreamSource.VERSION
|
||||
import org.apache.spark.sql.hudi.streaming.HoodieSourceOffset.INIT_OFFSET
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.catalyst.InternalRow
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder
|
||||
import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, Offset, Source}
|
||||
import org.apache.spark.sql.sources.Filter
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
|
||||
/**
|
||||
* The Struct Stream Source for Hudi to consume the data by streaming job.
|
||||
* @param sqlContext
|
||||
* @param metadataPath
|
||||
* @param schemaOption
|
||||
* @param parameters
|
||||
*/
|
||||
class HoodieStreamSource(
|
||||
sqlContext: SQLContext,
|
||||
metadataPath: String,
|
||||
schemaOption: Option[StructType],
|
||||
parameters: Map[String, String])
|
||||
extends Source with Logging with Serializable with SparkAdapterSupport {
|
||||
|
||||
@transient private val hadoopConf = sqlContext.sparkSession.sessionState.newHadoopConf()
|
||||
private lazy val tablePath: Path = {
|
||||
val path = new Path(parameters.getOrElse("path", "Missing 'path' option"))
|
||||
val fs = path.getFileSystem(hadoopConf)
|
||||
TablePathUtils.getTablePath(fs, path).get()
|
||||
}
|
||||
private lazy val metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(tablePath.toString).build()
|
||||
private lazy val tableType = metaClient.getTableType
|
||||
|
||||
@transient private var lastOffset: HoodieSourceOffset = _
|
||||
@transient private lazy val initialOffsets = {
|
||||
val metadataLog =
|
||||
new HDFSMetadataLog[HoodieSourceOffset](sqlContext.sparkSession, metadataPath) {
|
||||
override def serialize(metadata: HoodieSourceOffset, out: OutputStream): Unit = {
|
||||
val writer = new BufferedWriter(new OutputStreamWriter(out, StandardCharsets.UTF_8))
|
||||
writer.write("v" + VERSION + "\n")
|
||||
writer.write(metadata.json)
|
||||
writer.flush()
|
||||
}
|
||||
|
||||
/**
|
||||
* Deserialize the init offset from the metadata file.
|
||||
* The format in the metadata file is like this:
|
||||
* ----------------------------------------------
|
||||
* v1 -- The version info in the first line
|
||||
* offsetJson -- The json string of HoodieSourceOffset in the rest of the file
|
||||
* -----------------------------------------------
|
||||
* @param in
|
||||
* @return
|
||||
*/
|
||||
override def deserialize(in: InputStream): HoodieSourceOffset = {
|
||||
val content = FileIOUtils.readAsUTFString(in)
|
||||
// Get version from the first line
|
||||
val firstLineEnd = content.indexOf("\n")
|
||||
if (firstLineEnd > 0) {
|
||||
val version = getVersion(content.substring(0, firstLineEnd))
|
||||
if (version > VERSION) {
|
||||
throw new IllegalStateException(s"UnSupportVersion: max support version is: $VERSION" +
|
||||
s" current version is: $version")
|
||||
}
|
||||
// Get offset from the rest line in the file
|
||||
HoodieSourceOffset.fromJson(content.substring(firstLineEnd + 1))
|
||||
} else {
|
||||
throw new IllegalStateException(s"Bad metadata format, failed to find the version line.")
|
||||
}
|
||||
}
|
||||
}
|
||||
metadataLog.get(0).getOrElse {
|
||||
metadataLog.add(0, INIT_OFFSET)
|
||||
INIT_OFFSET
|
||||
}
|
||||
}
|
||||
|
||||
private def getVersion(versionLine: String): Int = {
|
||||
if (versionLine.startsWith("v")) {
|
||||
versionLine.substring(1).toInt
|
||||
} else {
|
||||
throw new IllegalStateException(s"Illegal version line: $versionLine " +
|
||||
s"in the streaming metadata path")
|
||||
}
|
||||
}
|
||||
|
||||
override def schema: StructType = {
|
||||
schemaOption.getOrElse {
|
||||
val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(schemaUtil.getTableAvroSchema)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest offset from the hoodie table.
|
||||
* @return
|
||||
*/
|
||||
override def getOffset: Option[Offset] = {
|
||||
metaClient.reloadActiveTimeline()
|
||||
val activeInstants = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
|
||||
if (!activeInstants.empty()) {
|
||||
val currentLatestCommitTime = activeInstants.lastInstant().get().getTimestamp
|
||||
if (lastOffset == null || currentLatestCommitTime > lastOffset.commitTime) {
|
||||
lastOffset = HoodieSourceOffset(currentLatestCommitTime)
|
||||
}
|
||||
} else { // if there are no active commits, use the init offset
|
||||
lastOffset = initialOffsets
|
||||
}
|
||||
Some(lastOffset)
|
||||
}
|
||||
|
||||
override def getBatch(start: Option[Offset], end: Offset): DataFrame = {
|
||||
initialOffsets
|
||||
|
||||
val startOffset = start.map(HoodieSourceOffset(_))
|
||||
.getOrElse(initialOffsets)
|
||||
val endOffset = HoodieSourceOffset(end)
|
||||
|
||||
if (startOffset == endOffset) {
|
||||
sqlContext.internalCreateDataFrame(
|
||||
sqlContext.sparkContext.emptyRDD[InternalRow].setName("empty"), schema, isStreaming = true)
|
||||
} else {
|
||||
// Consume the data between (startCommitTime, endCommitTime]
|
||||
val incParams = parameters ++ Map(
|
||||
DataSourceReadOptions.BEGIN_INSTANTTIME.key -> startCommitTime(startOffset),
|
||||
DataSourceReadOptions.END_INSTANTTIME.key -> endOffset.commitTime
|
||||
)
|
||||
|
||||
val rdd = tableType match {
|
||||
case HoodieTableType.COPY_ON_WRITE =>
|
||||
val serDe = sparkAdapter.createSparkRowSerDe(RowEncoder(schema))
|
||||
new IncrementalRelation(sqlContext, incParams, schema, metaClient)
|
||||
.buildScan()
|
||||
.map(serDe.serializeRow)
|
||||
case HoodieTableType.MERGE_ON_READ =>
|
||||
val requiredColumns = schema.fields.map(_.name)
|
||||
new MergeOnReadIncrementalRelation(sqlContext, incParams, schema, metaClient)
|
||||
.buildScan(requiredColumns, Array.empty[Filter])
|
||||
.asInstanceOf[RDD[InternalRow]]
|
||||
case _ => throw new IllegalArgumentException(s"UnSupport tableType: $tableType")
|
||||
}
|
||||
sqlContext.internalCreateDataFrame(rdd, schema, isStreaming = true)
|
||||
}
|
||||
}
|
||||
|
||||
private def startCommitTime(startOffset: HoodieSourceOffset): String = {
|
||||
startOffset match {
|
||||
case INIT_OFFSET => startOffset.commitTime
|
||||
case HoodieSourceOffset(commitTime) =>
|
||||
val time = HoodieActiveTimeline.parseDateFromInstantTime(commitTime).getTime
|
||||
// As we consume the data between (start, end], start is not included,
|
||||
// so we +1s to the start commit time here.
|
||||
HoodieActiveTimeline.formatDate(new Date(time + 1000))
|
||||
case _=> throw new IllegalStateException("UnKnow offset type.")
|
||||
}
|
||||
}
|
||||
|
||||
override def stop(): Unit = {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
object HoodieStreamSource {
|
||||
val VERSION = 1
|
||||
}
|
||||
Reference in New Issue
Block a user