1
0

[HUDI-3563] Make quickstart examples covered by CI tests (#5082)

This commit is contained in:
ForwardXu
2022-03-25 16:37:17 +08:00
committed by GitHub
parent f20c9867d7
commit e5c3f9089b
38 changed files with 2980 additions and 225 deletions

View File

@@ -0,0 +1,39 @@
/*
* 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.examples.common;
import org.apache.avro.Schema;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.spark.api.java.JavaSparkContext;
/**
* the example SchemaProvider of example json data from uber.
*/
public class ExampleDataSchemaProvider extends SchemaProvider {
public ExampleDataSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
}
@Override
public Schema getSourceSchema() {
return HoodieExampleDataGenerator.avroSchema;
}
}

View File

@@ -0,0 +1,60 @@
/*
* 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.examples.common;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.SparkSession;
import java.util.HashMap;
import java.util.Map;
/**
* Bunch of util methods.
*/
public class HoodieExampleSparkUtils {
private static Map<String, String> defaultConf() {
Map<String, String> additionalConfigs = new HashMap<>();
additionalConfigs.put("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
additionalConfigs.put("spark.kryoserializer.buffer.max", "512m");
return additionalConfigs;
}
public static SparkConf defaultSparkConf(String appName) {
return buildSparkConf(appName, defaultConf());
}
public static SparkConf buildSparkConf(String appName, Map<String, String> additionalConfigs) {
SparkConf sparkConf = new SparkConf().setAppName(appName);
additionalConfigs.forEach(sparkConf::set);
return sparkConf;
}
public static SparkSession defaultSparkSession(String appName) {
return buildSparkSession(appName, defaultConf());
}
public static SparkSession buildSparkSession(String appName, Map<String, String> additionalConfigs) {
SparkSession.Builder builder = SparkSession.builder().appName(appName);
additionalConfigs.forEach(builder::config);
return builder.getOrCreate();
}
}

View File

@@ -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.hudi.examples.common;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.utilities.transform.Transformer;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
/**
* Identity transformer.
*/
public class IdentityTransformer implements Transformer {
@Override
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset<Row> rowDataset,
TypedProperties properties) {
return rowDataset;
}
}

View File

@@ -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.hudi.examples.common;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hudi.utilities.sources.InputBatch;
import org.apache.hudi.utilities.sources.JsonSource;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
import java.util.List;
public class RandomJsonSource extends JsonSource {
private HoodieExampleDataGenerator<HoodieAvroPayload> dataGen;
public RandomJsonSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession, SchemaProvider schemaProvider) {
super(props, sparkContext, sparkSession, schemaProvider);
dataGen = new HoodieExampleDataGenerator<>();
}
protected InputBatch<JavaRDD<String>> fetchNewData(Option<String> lastCkptStr, long sourceLimit) {
String commitTime = HoodieActiveTimeline.createNewInstantTime();
List<String> inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20));
return new InputBatch<>(Option.of(sparkContext.parallelize(inserts, 1)), commitTime);
}
}

View File

@@ -0,0 +1,227 @@
/*
* 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.examples.quickstart;
import static org.apache.hudi.config.HoodieWriteConfig.TBL_NAME;
import static org.apache.spark.sql.SaveMode.Append;
import static org.apache.spark.sql.SaveMode.Overwrite;
import java.util.List;
import org.apache.commons.lang.ArrayUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.examples.common.HoodieExampleDataGenerator;
import org.apache.hudi.examples.common.HoodieExampleSparkUtils;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SparkSession;
public final class HoodieSparkQuickstart {
private HoodieSparkQuickstart() {
}
public static void main(String[] args) {
if (args.length < 2) {
System.err.println("Usage: HoodieWriteClientExample <tablePath> <tableName>");
System.exit(1);
}
String tablePath = args[0];
String tableName = args[1];
SparkSession spark = HoodieExampleSparkUtils.defaultSparkSession("Hudi Spark basic example");
SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example");
try (JavaSparkContext jsc = new JavaSparkContext(sparkConf)) {
final HoodieExampleDataGenerator<HoodieAvroPayload> dataGen = new HoodieExampleDataGenerator<>();
insertData(spark, jsc, tablePath, tableName, dataGen);
updateData(spark, jsc, tablePath, tableName, dataGen);
queryData(spark, jsc, tablePath, tableName, dataGen);
incrementalQuery(spark, tablePath, tableName);
pointInTimeQuery(spark, tablePath, tableName);
delete(spark, tablePath, tableName);
deleteByPartition(spark, tablePath, tableName);
}
}
/**
* Generate some new trips, load them into a DataFrame and write the DataFrame into the Hudi dataset as below.
*/
public static void insertData(SparkSession spark, JavaSparkContext jsc, String tablePath, String tableName,
HoodieExampleDataGenerator<HoodieAvroPayload> dataGen) {
String commitTime = Long.toString(System.currentTimeMillis());
List<String> inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20));
Dataset<Row> df = spark.read().json(jsc.parallelize(inserts, 1));
df.write().format("org.apache.hudi").
options(QuickstartUtils.getQuickstartWriteConfigs()).
option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts").
option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid").
option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath").
option(TBL_NAME.key(), tableName).
mode(Overwrite).
save(tablePath);
}
/**
* Load the data files into a DataFrame.
*/
public static void queryData(SparkSession spark, JavaSparkContext jsc, String tablePath, String tableName,
HoodieExampleDataGenerator<HoodieAvroPayload> dataGen) {
Dataset<Row> roViewDF = spark.
read().
format("org.apache.hudi").
load(tablePath + "/*/*/*/*");
roViewDF.createOrReplaceTempView("hudi_ro_table");
spark.sql("select fare, begin_lon, begin_lat, ts from hudi_ro_table where fare > 20.0").show();
// +-----------------+-------------------+-------------------+---+
// | fare| begin_lon| begin_lat| ts|
// +-----------------+-------------------+-------------------+---+
// |98.88075495133515|0.39556048623031603|0.17851135255091155|0.0|
// ...
spark.sql(
"select _hoodie_commit_time, _hoodie_record_key, _hoodie_partition_path, rider, driver, fare from hudi_ro_table")
.show();
// +-------------------+--------------------+----------------------+-------------------+--------------------+------------------+
// |_hoodie_commit_time| _hoodie_record_key|_hoodie_partition_path| rider| driver| fare|
// +-------------------+--------------------+----------------------+-------------------+--------------------+------------------+
// | 20191231181501|31cafb9f-0196-4b1...| 2020/01/02|rider-1577787297889|driver-1577787297889| 98.88075495133515|
// ...
}
/**
* This is similar to inserting new data. Generate updates to existing trips using the data generator,
* load into a DataFrame and write DataFrame into the hudi dataset.
*/
public static void updateData(SparkSession spark, JavaSparkContext jsc, String tablePath, String tableName,
HoodieExampleDataGenerator<HoodieAvroPayload> dataGen) {
String commitTime = Long.toString(System.currentTimeMillis());
List<String> updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10));
Dataset<Row> df = spark.read().json(jsc.parallelize(updates, 1));
df.write().format("org.apache.hudi").
options(QuickstartUtils.getQuickstartWriteConfigs()).
option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts").
option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid").
option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath").
option(TBL_NAME.key(), tableName).
mode(Append).
save(tablePath);
}
/**
* Deleta data based in data information.
*/
public static void delete(SparkSession spark, String tablePath, String tableName) {
Dataset<Row> roViewDF = spark.read().format("org.apache.hudi").load(tablePath + "/*/*/*/*");
roViewDF.createOrReplaceTempView("hudi_ro_table");
Dataset<Row> df = spark.sql("select uuid, partitionpath, ts from hudi_ro_table limit 2");
df.write().format("org.apache.hudi").
options(QuickstartUtils.getQuickstartWriteConfigs()).
option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts").
option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "uuid").
option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath").
option(TBL_NAME.key(), tableName).
option("hoodie.datasource.write.operation", WriteOperationType.DELETE.value()).
mode(Append).
save(tablePath);
}
/**
* Delete the data of a single or multiple partitions.
*/
public static void deleteByPartition(SparkSession spark, String tablePath, String tableName) {
Dataset<Row> df = spark.emptyDataFrame();
df.write().format("org.apache.hudi").
options(QuickstartUtils.getQuickstartWriteConfigs()).
option(HoodieWriteConfig.PRECOMBINE_FIELD_NAME.key(), "ts").
option(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), "uuid").
option(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partitionpath").
option(TBL_NAME.key(), tableName).
option("hoodie.datasource.write.operation", WriteOperationType.DELETE.value()).
option("hoodie.datasource.write.partitions.to.delete",
ArrayUtils.toString(HoodieExampleDataGenerator.DEFAULT_PARTITION_PATHS, ",")).
mode(Append).
save(tablePath);
}
/**
* Hudi also provides capability to obtain a stream of records that changed since given commit timestamp.
* This can be achieved using Hudis incremental view and providing a begin time from which changes need to be streamed.
* We do not need to specify endTime, if we want all changes after the given commit (as is the common case).
*/
public static void incrementalQuery(SparkSession spark, String tablePath, String tableName) {
List<String> commits =
spark.sql("select distinct(_hoodie_commit_time) as commitTime from hudi_ro_table order by commitTime")
.toJavaRDD()
.map((Function<Row, String>) row -> row.getString(0))
.take(50);
String beginTime = commits.get(commits.size() - 2); // commit time we are interested in
// incrementally query data
Dataset<Row> incViewDF = spark.
read().
format("org.apache.hudi").
option("hoodie.datasource.query.type", "incremental").
option("hoodie.datasource.read.begin.instanttime", beginTime).
load(tablePath);
incViewDF.createOrReplaceTempView("hudi_incr_table");
spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0")
.show();
}
/**
* Lets look at how to query data as of a specific time.
* The specific time can be represented by pointing endTime to a specific commit time
* and beginTime to “000” (denoting earliest possible commit time).
*/
public static void pointInTimeQuery(SparkSession spark, String tablePath, String tableName) {
List<String> commits =
spark.sql("select distinct(_hoodie_commit_time) as commitTime from hudi_ro_table order by commitTime")
.toJavaRDD()
.map((Function<Row, String>) row -> row.getString(0))
.take(50);
String beginTime = "000"; // Represents all commits > this time.
String endTime = commits.get(commits.size() - 2); // commit time we are interested in
//incrementally query data
Dataset<Row> incViewDF = spark.read().format("org.apache.hudi").
option("hoodie.datasource.query.type", "incremental").
option("hoodie.datasource.read.begin.instanttime", beginTime).
option("hoodie.datasource.read.end.instanttime", endTime).
load(tablePath);
incViewDF.createOrReplaceTempView("hudi_incr_table");
spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0")
.show();
}
}

View File

@@ -0,0 +1,249 @@
/*
* 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.examples.quickstart;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
/**
* Class to be used in quickstart guide for generating inserts and updates against a corpus. Test data uses a toy Uber
* trips, data model.
*/
public class QuickstartUtils {
public static class DataGenerator {
private static final String DEFAULT_FIRST_PARTITION_PATH = "americas/united_states/san_francisco";
private static final String DEFAULT_SECOND_PARTITION_PATH = "americas/brazil/sao_paulo";
private static final String DEFAULT_THIRD_PARTITION_PATH = "asia/india/chennai";
private static final String[] DEFAULT_PARTITION_PATHS =
{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH, DEFAULT_THIRD_PARTITION_PATH};
static String TRIP_EXAMPLE_SCHEMA = "{\"type\": \"record\",\"name\": \"triprec\",\"fields\": [ "
+ "{\"name\": \"ts\",\"type\": \"long\"},{\"name\": \"uuid\", \"type\": \"string\"},"
+ "{\"name\": \"rider\", \"type\": \"string\"},{\"name\": \"driver\", \"type\": \"string\"},"
+ "{\"name\": \"begin_lat\", \"type\": \"double\"},{\"name\": \"begin_lon\", \"type\": \"double\"},"
+ "{\"name\": \"end_lat\", \"type\": \"double\"},{\"name\": \"end_lon\", \"type\": \"double\"},"
+ "{\"name\":\"fare\",\"type\": \"double\"}]}";
static Schema avroSchema = new Schema.Parser().parse(TRIP_EXAMPLE_SCHEMA);
private static Random rand = new Random(46474747);
private final Map<Integer, HoodieKey> existingKeys;
private final String[] partitionPaths;
private int numExistingKeys;
public DataGenerator() {
this(DEFAULT_PARTITION_PATHS, new HashMap<>());
}
public DataGenerator(String[] partitionPaths) {
this(partitionPaths, new HashMap<>());
}
private DataGenerator(String[] partitionPaths, Map<Integer, HoodieKey> keyPartitionMap) {
this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length);
this.existingKeys = keyPartitionMap;
}
private static String generateRandomString() {
int leftLimit = 48; // ascii for 0
int rightLimit = 57; // ascii for 9
int stringLength = 3;
StringBuilder buffer = new StringBuilder(stringLength);
for (int i = 0; i < stringLength; i++) {
int randomLimitedInt = leftLimit + (int) (rand.nextFloat() * (rightLimit - leftLimit + 1));
buffer.append((char) randomLimitedInt);
}
return buffer.toString();
}
public int getNumExistingKeys() {
return numExistingKeys;
}
public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName,
long timestamp) {
GenericRecord rec = new GenericData.Record(avroSchema);
rec.put("uuid", rowKey);
rec.put("ts", timestamp);
rec.put("rider", riderName);
rec.put("driver", driverName);
rec.put("begin_lat", rand.nextDouble());
rec.put("begin_lon", rand.nextDouble());
rec.put("end_lat", rand.nextDouble());
rec.put("end_lon", rand.nextDouble());
rec.put("fare", rand.nextDouble() * 100);
return rec;
}
/**
* Generates a new avro record of the above schema format, retaining the key if optionally provided. The
* riderDriverSuffix string is a random String to simulate updates by changing the rider driver fields for records
* belonging to the same commit. It is purely used for demo purposes. In real world, the actual updates are assumed
* to be provided based on the application requirements.
*/
public static OverwriteWithLatestAvroPayload generateRandomValue(HoodieKey key, String riderDriverSuffix)
throws IOException {
// The timestamp generated is limited to range from 7 days before to now, to avoid generating too many
// partitionPaths when user use timestamp as partitionPath filed.
GenericRecord rec =
generateGenericRecord(key.getRecordKey(), "rider-" + riderDriverSuffix, "driver-"
+ riderDriverSuffix, generateRangeRandomTimestamp(7));
return new OverwriteWithLatestAvroPayload(Option.of(rec));
}
/**
* Generate timestamp range from {@param daysTillNow} before to now.
*/
private static long generateRangeRandomTimestamp(int daysTillNow) {
long maxIntervalMillis = daysTillNow * 24 * 60 * 60 * 1000L;
return System.currentTimeMillis() - (long)(Math.random() * maxIntervalMillis);
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public Stream<HoodieRecord> generateInsertsStream(String randomString, Integer n) {
int currSize = getNumExistingKeys();
return IntStream.range(0, n).boxed().map(i -> {
String partitionPath = partitionPaths[rand.nextInt(partitionPaths.length)];
HoodieKey key = new HoodieKey(UUID.randomUUID().toString(), partitionPath);
existingKeys.put(currSize + i, key);
numExistingKeys++;
try {
return new HoodieAvroRecord(key, generateRandomValue(key, randomString));
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
}
/**
* Generates new inserts, uniformly across the partition paths above. It also updates the list of existing keys.
*/
public List<HoodieRecord> generateInserts(Integer n) throws IOException {
String randomString = generateRandomString();
return generateInsertsStream(randomString, n).collect(Collectors.toList());
}
public HoodieRecord generateUpdateRecord(HoodieKey key, String randomString) throws IOException {
return new HoodieAvroRecord(key, generateRandomValue(key, randomString));
}
/**
* Generates new updates, randomly distributed across the keys above. There can be duplicates within the returned
* list
*
* @param n Number of updates (including dups)
* @return list of hoodie record updates
*/
public List<HoodieRecord> generateUpdates(Integer n) {
if (numExistingKeys == 0) {
throw new HoodieException("Data must have been written before performing the update operation");
}
String randomString = generateRandomString();
return IntStream.range(0, n).boxed().map(x -> {
try {
return generateUpdateRecord(existingKeys.get(rand.nextInt(numExistingKeys)), randomString);
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
}).collect(Collectors.toList());
}
/**
* Generates delete records for the passed in rows.
*
* @param rows List of {@link Row}s for which delete record need to be generated
* @return list of hoodie records to delete
*/
public List<String> generateDeletes(List<Row> rows) {
// if row.length() == 2, then the record contains "uuid" and "partitionpath" fields, otherwise,
// another field "ts" is available
return rows.stream().map(row -> row.length() == 2
? convertToString(row.getAs("uuid"), row.getAs("partitionpath"), null) :
convertToString(row.getAs("uuid"), row.getAs("partitionpath"), row.getAs("ts"))
).filter(os -> os.isPresent()).map(os -> os.get())
.collect(Collectors.toList());
}
public void close() {
existingKeys.clear();
}
}
private static Option<String> convertToString(HoodieRecord record) {
try {
String str = HoodieAvroUtils
.bytesToAvro(((OverwriteWithLatestAvroPayload) record.getData()).recordBytes, DataGenerator.avroSchema)
.toString();
str = "{" + str.substring(str.indexOf("\"ts\":"));
return Option.of(str.replaceAll("}", ", \"partitionpath\": \"" + record.getPartitionPath() + "\"}"));
} catch (IOException e) {
return Option.empty();
}
}
private static Option<String> convertToString(String uuid, String partitionPath, Long ts) {
StringBuffer stringBuffer = new StringBuffer();
stringBuffer.append("{");
stringBuffer.append("\"ts\": \"" + (ts == null ? "0.0" : ts) + "\",");
stringBuffer.append("\"uuid\": \"" + uuid + "\",");
stringBuffer.append("\"partitionpath\": \"" + partitionPath + "\"");
stringBuffer.append("}");
return Option.of(stringBuffer.toString());
}
public static List<String> convertToStringList(List<HoodieRecord> records) {
return records.stream().map(hr -> convertToString(hr)).filter(os -> os.isPresent()).map(os -> os.get())
.collect(Collectors.toList());
}
public static Map<String, String> getQuickstartWriteConfigs() {
Map<String, String> demoConfigs = new HashMap<>();
demoConfigs.put("hoodie.insert.shuffle.parallelism", "2");
demoConfigs.put("hoodie.upsert.shuffle.parallelism", "2");
demoConfigs.put("hoodie.bulkinsert.shuffle.parallelism", "2");
demoConfigs.put("hoodie.delete.shuffle.parallelism", "2");
return demoConfigs;
}
}

View File

@@ -0,0 +1,72 @@
/*
* 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.examples.spark;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.config.HoodieBootstrapConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.examples.common.HoodieExampleSparkUtils;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.spark.SparkConf;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.Dataset;
public class HoodieSparkBootstrapExample {
private static String tableType = HoodieTableType.MERGE_ON_READ.name();
public static void main(String[] args) throws Exception {
if (args.length < 5) {
System.err.println("Usage: HoodieSparkBootstrapExample <recordKey> <tableName> <partitionPath> <preCombineField> <basePath>");
System.exit(1);
}
String recordKey = args[0];
String tableName = args[1];
String partitionPath = args[2];
String preCombineField = args[3];
String basePath = args[4];
SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example");
SparkSession spark = SparkSession
.builder()
.appName("Java Spark SQL basic example")
.config("spark.some.config.option", "some-value")
.enableHiveSupport()
.getOrCreate();
Dataset df = spark.emptyDataFrame();
df.write().format("hudi").option(HoodieWriteConfig.TBL_NAME.key(), tableName)
.option(DataSourceWriteOptions.OPERATION().key(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL())
.option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKey)
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), partitionPath)
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), preCombineField)
.option(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieFileFormat.ORC.name())
.option(HoodieBootstrapConfig.BASE_PATH.key(), basePath)
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key(), NonpartitionedKeyGenerator.class.getCanonicalName())
.mode(SaveMode.Overwrite).save("/hudi/" + tableName);
df.count();
}
}

View File

@@ -0,0 +1,152 @@
/*
* 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.examples.spark;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.examples.common.HoodieExampleDataGenerator;
import org.apache.hudi.examples.common.HoodieExampleSparkUtils;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
/**
* Simple examples of #{@link SparkRDDWriteClient}.
*
* To run this example, you should
* <pre>
* 1. For running in IDE, set VM options `-Dspark.master=local[2]`;
* 2. For running in shell, using `spark-submit`.
*</pre>
*
* Usage: HoodieWriteClientExample <tablePath> <tableName>
* <tablePath> and <tableName> describe root path of hudi and table name
* for example, `HoodieWriteClientExample file:///tmp/hoodie/sample-table hoodie_rt`
*/
public class HoodieWriteClientExample {
private static final Logger LOG = LogManager.getLogger(HoodieWriteClientExample.class);
private static String tableType = HoodieTableType.COPY_ON_WRITE.name();
public static void main(String[] args) throws Exception {
if (args.length < 2) {
System.err.println("Usage: HoodieWriteClientExample <tablePath> <tableName>");
System.exit(1);
}
String tablePath = args[0];
String tableName = args[1];
SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example");
try (JavaSparkContext jsc = new JavaSparkContext(sparkConf)) {
// Generator of some records to be loaded in.
HoodieExampleDataGenerator<HoodieAvroPayload> dataGen = new HoodieExampleDataGenerator<>();
// initialize the table, if not done already
Path path = new Path(tablePath);
FileSystem fs = FSUtils.getFs(tablePath, jsc.hadoopConfiguration());
if (!fs.exists(path)) {
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(tableType)
.setTableName(tableName)
.setPayloadClass(HoodieAvroPayload.class)
.initTable(jsc.hadoopConfiguration(), tablePath);
}
// Create the write client to write some records in
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2).forTable(tableName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(20, 30).build()).build();
SparkRDDWriteClient<HoodieAvroPayload> client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg);
// inserts
String newCommitTime = client.startCommit();
LOG.info("Starting commit " + newCommitTime);
List<HoodieRecord<HoodieAvroPayload>> records = dataGen.generateInserts(newCommitTime, 10);
List<HoodieRecord<HoodieAvroPayload>> recordsSoFar = new ArrayList<>(records);
JavaRDD<HoodieRecord<HoodieAvroPayload>> writeRecords = jsc.parallelize(records, 1);
client.insert(writeRecords, newCommitTime);
// updates
newCommitTime = client.startCommit();
LOG.info("Starting commit " + newCommitTime);
List<HoodieRecord<HoodieAvroPayload>> toBeUpdated = dataGen.generateUpdates(newCommitTime, 2);
records.addAll(toBeUpdated);
recordsSoFar.addAll(toBeUpdated);
writeRecords = jsc.parallelize(records, 1);
client.upsert(writeRecords, newCommitTime);
// Delete
newCommitTime = client.startCommit();
LOG.info("Starting commit " + newCommitTime);
// just delete half of the records
int numToDelete = recordsSoFar.size() / 2;
List<HoodieKey> toBeDeleted = recordsSoFar.stream().map(HoodieRecord::getKey).limit(numToDelete).collect(Collectors.toList());
JavaRDD<HoodieKey> deleteRecords = jsc.parallelize(toBeDeleted, 1);
client.delete(deleteRecords, newCommitTime);
// Delete by partition
newCommitTime = client.startCommit();
client.startCommitWithTime(newCommitTime, HoodieTimeline.REPLACE_COMMIT_ACTION);
LOG.info("Starting commit " + newCommitTime);
// The partition where the data needs to be deleted
List<String> partitionList = toBeDeleted.stream().map(s -> s.getPartitionPath()).distinct().collect(Collectors.toList());
List<String> deleteList = recordsSoFar.stream().filter(f -> !partitionList.contains(f.getPartitionPath()))
.map(m -> m.getKey().getPartitionPath()).distinct().collect(Collectors.toList());
client.deletePartitions(deleteList, newCommitTime);
// compaction
if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) {
Option<String> instant = client.scheduleCompaction(Option.empty());
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = client.compact(instant.get());
client.commitCompaction(instant.get(), compactionMetadata.getCommitMetadata().get(), Option.empty());
}
}
}
}

View File

@@ -0,0 +1,10 @@
{"ts": 0.0, "uuid": "d4b5b1eb-5c4f-484b-9f5b-d9e0c29ab0ba", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.18433112391820694, "begin_lon": 0.4457079093559174, "end_lat": 0.38128402026859787, "end_lon": 0.4528353922784837, "fare": 18.769410203570114, "partitionpath": "2020/01/02"}
{"ts": 0.0, "uuid": "12a6af4a-08e9-4fbd-8942-32525e8ffc25", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.3415828471638285, "begin_lon": 0.35472417786727917, "end_lat": 0.5744827818563615, "end_lon": 0.4645148864505745, "fare": 83.12040940386028, "partitionpath": "2020/01/01"}
{"ts": 0.0, "uuid": "209289a5-fe6d-42ba-92c3-3380c96d382e", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.434923811219014, "begin_lon": 0.834448733526223, "end_lat": 0.2861201538495416, "end_lon": 0.6434040231985722, "fare": 12.289323546303788, "partitionpath": "2020/01/01"}
{"ts": 0.0, "uuid": "2e8a6c33-ecc3-4279-ac4b-9571ecfb2fc3", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.6427696295884016, "begin_lon": 0.23849882507684073, "end_lat": 0.6613489410705939, "end_lon": 0.6220454661413275, "fare": 72.024792576853, "partitionpath": "2020/01/01"}
{"ts": 0.0, "uuid": "2acc3303-86d1-4f73-a062-cd05faf3c46a", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.9356018115236618, "begin_lon": 0.44714286038480855, "end_lat": 0.8473860258521023, "end_lon": 0.1762368947074756, "fare": 79.42627821413218, "partitionpath": "2020/01/02"}
{"ts": 0.0, "uuid": "25c579d6-e480-4373-ae71-b4d524a74142", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.17851135255091155, "begin_lon": 0.39556048623031603, "end_lat": 0.5878106318835439, "end_lon": 0.36519521355305173, "fare": 98.88075495133515, "partitionpath": "2020/01/02"}
{"ts": 0.0, "uuid": "974198b6-507e-4de2-bbe3-997d025e83b9", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.19179139106643606, "begin_lon": 0.8157865134723307, "end_lat": 0.6418467159488594, "end_lon": 0.14516349705850584, "fare": 12.153670568058683, "partitionpath": "2020/01/01"}
{"ts": 0.0, "uuid": "f91c2067-7527-491b-a766-e180d6da1371", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.33922164839486424, "begin_lon": 0.909372837469859, "end_lat": 0.9017656600243008, "end_lon": 0.8236411667430927, "fare": 2.0856583634078385, "partitionpath": "2020/01/03"}
{"ts": 0.0, "uuid": "1b9fb778-3349-4172-b81d-522e181b36e1", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.6662084366450246, "begin_lon": 0.9065078444936647, "end_lat": 0.7124299678100179, "end_lon": 0.05336723040266267, "fare": 38.63372961020515, "partitionpath": "2020/01/02"}
{"ts": 0.0, "uuid": "a4b56c00-f372-414a-9c1c-458c10d648b9", "rider": "rider-20200113114823", "driver": "driver-20200113114823", "begin_lat": 0.4106290929046368, "begin_lon": 0.964603455586492, "end_lat": 0.13957566957654388, "end_lon": 0.45400191464227213, "fare": 81.37564420028626, "partitionpath": "2020/01/02"}

View File

@@ -0,0 +1,31 @@
###
# 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.
###
# Key fields, for kafka example
hoodie.datasource.write.recordkey.field=uuid
hoodie.datasource.write.partitionpath.field=driver
# Kafka Source topic
hoodie.deltastreamer.source.kafka.topic=hoodie-source-topic
# Kafka props
# The kafka cluster we want to ingest from
bootstrap.servers=localhost:9092
auto.offset.reset=earliest
group.id=hoodie-delta-streamer

View File

@@ -0,0 +1,211 @@
/*
* 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.examples.spark
import org.apache.hudi.DataSourceReadOptions.{BEGIN_INSTANTTIME, END_INSTANTTIME, QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL}
import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD, PARTITIONS_TO_DELETE, OPERATION, DELETE_PARTITION_OPERATION_OPT_VAL, DELETE_OPERATION_OPT_VAL}
import org.apache.hudi.examples.quickstart.QuickstartUtils.getQuickstartWriteConfigs
import org.apache.hudi.common.model.HoodieAvroPayload
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils}
import org.apache.spark.sql.SaveMode.{Append, Overwrite}
import org.apache.spark.sql.SparkSession
import scala.collection.JavaConversions._
/**
* Simple examples of [[org.apache.hudi.DefaultSource]]
*
* To run this example, you should
* 1. For running in IDE, set VM options `-Dspark.master=local[2]`
* 2. For running in shell, using `spark-submit`
*
* Usage: HoodieWriteClientExample <tablePath> <tableName>.
* <tablePath> and <tableName> describe root path of hudi and table name
* for example, `HoodieDataSourceExample file:///tmp/hoodie/hudi_cow_table hudi_cow_table`
*/
object HoodieDataSourceExample {
def main(args: Array[String]): Unit = {
if (args.length < 2) {
System.err.println("Usage: HoodieDataSourceExample <tablePath> <tableName>")
System.exit(1)
}
val tablePath = args(0)
val tableName = args(1)
val spark = HoodieExampleSparkUtils.defaultSparkSession("Hudi Spark basic example")
val dataGen = new HoodieExampleDataGenerator[HoodieAvroPayload]
insertData(spark, tablePath, tableName, dataGen)
updateData(spark, tablePath, tableName, dataGen)
queryData(spark, tablePath, tableName, dataGen)
incrementalQuery(spark, tablePath, tableName)
pointInTimeQuery(spark, tablePath, tableName)
delete(spark, tablePath, tableName)
deleteByPartition(spark, tablePath, tableName)
spark.stop()
}
/**
* Generate some new trips, load them into a DataFrame and write the DataFrame into the Hudi dataset as below.
*/
def insertData(spark: SparkSession, tablePath: String, tableName: String, dataGen: HoodieExampleDataGenerator[HoodieAvroPayload]): Unit = {
val commitTime: String = System.currentTimeMillis().toString
val inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20))
val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
mode(Overwrite).
save(tablePath)
}
/**
* Load the data files into a DataFrame.
*/
def queryData(spark: SparkSession, tablePath: String, tableName: String, dataGen: HoodieExampleDataGenerator[HoodieAvroPayload]): Unit = {
val roViewDF = spark.
read.
format("org.apache.hudi").
load(tablePath + "/*/*/*/*")
roViewDF.createOrReplaceTempView("hudi_ro_table")
spark.sql("select fare, begin_lon, begin_lat, ts from hudi_ro_table where fare > 20.0").show()
// +-----------------+-------------------+-------------------+---+
// | fare| begin_lon| begin_lat| ts|
// +-----------------+-------------------+-------------------+---+
// |98.88075495133515|0.39556048623031603|0.17851135255091155|0.0|
// ...
spark.sql("select _hoodie_commit_time, _hoodie_record_key, _hoodie_partition_path, rider, driver, fare from hudi_ro_table").show()
// +-------------------+--------------------+----------------------+-------------------+--------------------+------------------+
// |_hoodie_commit_time| _hoodie_record_key|_hoodie_partition_path| rider| driver| fare|
// +-------------------+--------------------+----------------------+-------------------+--------------------+------------------+
// | 20191231181501|31cafb9f-0196-4b1...| 2020/01/02|rider-1577787297889|driver-1577787297889| 98.88075495133515|
// ...
}
/**
* This is similar to inserting new data. Generate updates to existing trips using the data generator,
* load into a DataFrame and write DataFrame into the hudi dataset.
*/
def updateData(spark: SparkSession, tablePath: String, tableName: String, dataGen: HoodieExampleDataGenerator[HoodieAvroPayload]): Unit = {
val commitTime: String = System.currentTimeMillis().toString
val updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10))
val df = spark.read.json(spark.sparkContext.parallelize(updates, 1))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
mode(Append).
save(tablePath)
}
/**
* Deleta data based in data information.
*/
def delete(spark: SparkSession, tablePath: String, tableName: String): Unit = {
val roViewDF = spark.read.format("org.apache.hudi").load(tablePath + "/*/*/*/*")
roViewDF.createOrReplaceTempView("hudi_ro_table")
val df = spark.sql("select uuid, partitionpath, ts from hudi_ro_table limit 2")
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
option(OPERATION.key, DELETE_OPERATION_OPT_VAL).
mode(Append).
save(tablePath)
}
/**
* Delete the data of a single or multiple partitions.
*/
def deleteByPartition(spark: SparkSession, tablePath: String, tableName: String): Unit = {
val df = spark.emptyDataFrame
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
option(OPERATION.key, DELETE_PARTITION_OPERATION_OPT_VAL).
option(PARTITIONS_TO_DELETE.key(), HoodieExampleDataGenerator.DEFAULT_PARTITION_PATHS.mkString(",")).
mode(Append).
save(tablePath)
}
/**
* Hudi also provides capability to obtain a stream of records that changed since given commit timestamp.
* This can be achieved using Hudis incremental view and providing a begin time from which changes need to be streamed.
* We do not need to specify endTime, if we want all changes after the given commit (as is the common case).
*/
def incrementalQuery(spark: SparkSession, tablePath: String, tableName: String): Unit = {
import spark.implicits._
val commits = spark.sql("select distinct(_hoodie_commit_time) as commitTime from hudi_ro_table order by commitTime").map(k => k.getString(0)).take(50)
val beginTime = commits(commits.length - 2) // commit time we are interested in
// incrementally query data
val incViewDF = spark.
read.
format("org.apache.hudi").
option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL).
option(BEGIN_INSTANTTIME.key, beginTime).
load(tablePath)
incViewDF.createOrReplaceTempView("hudi_incr_table")
spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0").show()
}
/**
* Lets look at how to query data as of a specific time.
* The specific time can be represented by pointing endTime to a specific commit time
* and beginTime to “000” (denoting earliest possible commit time).
*/
def pointInTimeQuery(spark: SparkSession, tablePath: String, tableName: String): Unit = {
import spark.implicits._
val commits = spark.sql("select distinct(_hoodie_commit_time) as commitTime from hudi_ro_table order by commitTime").map(k => k.getString(0)).take(50)
val beginTime = "000" // Represents all commits > this time.
val endTime = commits(commits.length - 2) // commit time we are interested in
//incrementally query data
val incViewDF = spark.read.format("org.apache.hudi").
option(QUERY_TYPE.key, QUERY_TYPE_INCREMENTAL_OPT_VAL).
option(BEGIN_INSTANTTIME.key, beginTime).
option(END_INSTANTTIME.key, endTime).
load(tablePath)
incViewDF.createOrReplaceTempView("hudi_incr_table")
spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0").show()
}
}

View File

@@ -0,0 +1,114 @@
/*
* 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.examples.spark
import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD, TABLE_TYPE}
import org.apache.hudi.examples.quickstart.QuickstartUtils.getQuickstartWriteConfigs
import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.model.{HoodieAvroPayload, HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.util.Option
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils}
import org.apache.spark.sql.SaveMode.{Append, Overwrite}
import org.apache.spark.sql.SparkSession
import scala.collection.JavaConverters._
/**
* Simple example to run a compaction job for MOR table.
* To run this example, you should:
* 1. For running in IDE, set VM options `-Dspark.master=local[2]`
* 2. For running in shell, using `spark-submit`
*
* Usage: HoodieMorCompactionJob <tablePath> <tableName>.
* <tablePath> and <tableName> describe root path of hudi and table name
* for example, `HoodieMorCompactionJob file:///tmp/hoodie/hudi_mor_table hudi_mor_table`
*/
object HoodieMorCompactionJob {
def main(args: Array[String]): Unit = {
if (args.length < 2) {
System.err.println("Usage: HoodieMorCompactionJob <tablePath> <tableName>")
System.exit(1)
}
val spark = HoodieExampleSparkUtils.defaultSparkSession("Hudi MOR table compaction via Spark example")
val dataGen = new HoodieExampleDataGenerator[HoodieAvroPayload]
val tablePath = args(0)
val tableName = args(1)
insertData(spark, tablePath, tableName, dataGen, HoodieTableType.MERGE_ON_READ.name())
updateData(spark, tablePath, tableName, dataGen, HoodieTableType.MERGE_ON_READ.name())
val cfg = HoodieWriteConfig.newBuilder()
.withPath(tablePath)
.withSchema(HoodieExampleDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2)
.forTable(tableName)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withInlineCompaction(true)
.withMaxNumDeltaCommitsBeforeCompaction(1).build())
.build()
val client = new SparkRDDWriteClient[HoodieRecordPayload[Nothing]](new HoodieSparkEngineContext(spark.sparkContext), cfg)
try {
val instant = client.scheduleCompaction(Option.empty())
client.compact(instant.get())
client.clean()
} catch {
case e: Exception => System.err.println(s"Compaction failed due to", e)
} finally {
client.close()
spark.stop()
}
}
def insertData(spark: SparkSession, tablePath: String, tableName: String,
dataGen: HoodieExampleDataGenerator[HoodieAvroPayload], tableType: String): Unit = {
val commitTime: String = System.currentTimeMillis().toString
val inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20))
val df = spark.read.json(spark.sparkContext.parallelize(inserts.asScala, 1))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
option(TABLE_TYPE.key, tableType).
mode(Overwrite).
save(tablePath)
}
def updateData(spark: SparkSession, tablePath: String, tableName: String,
dataGen: HoodieExampleDataGenerator[HoodieAvroPayload], tableType: String): Unit = {
val commitTime: String = System.currentTimeMillis().toString
val updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10))
val df = spark.read.json(spark.sparkContext.parallelize(updates.asScala, 1))
df.write.format("org.apache.hudi").
options(getQuickstartWriteConfigs).
option(PRECOMBINE_FIELD.key, "ts").
option(RECORDKEY_FIELD.key, "uuid").
option(PARTITIONPATH_FIELD.key, "partitionpath").
option(TBL_NAME.key, tableName).
option(TABLE_TYPE.key, tableType).
mode(Append).
save(tablePath)
}
}