HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
@@ -0,0 +1,277 @@
|
||||
/*
|
||||
* 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.utilities;
|
||||
|
||||
import com.beust.jcommander.IValueValidator;
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.beust.jcommander.ParameterException;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieJsonPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetInputFormat;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Loads data from Parquet Sources
|
||||
*/
|
||||
public class HDFSParquetImporter implements Serializable {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(HDFSParquetImporter.class);
|
||||
|
||||
public static final SimpleDateFormat PARTITION_FORMATTER = new SimpleDateFormat("yyyy/MM/dd");
|
||||
private static volatile Logger logger = LogManager.getLogger(HDFSParquetImporter.class);
|
||||
private final Config cfg;
|
||||
private transient FileSystem fs;
|
||||
/**
|
||||
* Bag of properties with source, hoodie client, key generator etc.
|
||||
*/
|
||||
private TypedProperties props;
|
||||
|
||||
public HDFSParquetImporter(Config cfg) throws IOException {
|
||||
this.cfg = cfg;
|
||||
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) :
|
||||
UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
||||
log.info("Creating Cleaner with configs : " + props.toString());
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
HDFSParquetImporter dataImporter = new HDFSParquetImporter(cfg);
|
||||
JavaSparkContext jssc = UtilHelpers
|
||||
.buildSparkContext("data-importer-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory);
|
||||
try {
|
||||
dataImporter.dataImport(jssc, cfg.retry);
|
||||
} finally {
|
||||
jssc.stop();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public int dataImport(JavaSparkContext jsc, int retry) throws Exception {
|
||||
this.fs = FSUtils.getFs(cfg.targetPath, jsc.hadoopConfiguration());
|
||||
int ret = -1;
|
||||
try {
|
||||
// Verify that targetPath is not present.
|
||||
if (fs.exists(new Path(cfg.targetPath))) {
|
||||
throw new HoodieIOException(String.format("Make sure %s is not present.", cfg.targetPath));
|
||||
}
|
||||
do {
|
||||
ret = dataImport(jsc);
|
||||
} while (ret != 0 && retry-- > 0);
|
||||
} catch (Throwable t) {
|
||||
logger.error(t);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected int dataImport(JavaSparkContext jsc) throws IOException {
|
||||
try {
|
||||
if (fs.exists(new Path(cfg.targetPath))) {
|
||||
// cleanup target directory.
|
||||
fs.delete(new Path(cfg.targetPath), true);
|
||||
}
|
||||
|
||||
//Get schema.
|
||||
String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
|
||||
|
||||
// Initialize target hoodie table.
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, cfg.tableName);
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, cfg.tableType);
|
||||
HoodieTableMetaClient
|
||||
.initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties);
|
||||
|
||||
HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr,
|
||||
cfg.parallelism, Option.empty(), props);
|
||||
|
||||
JavaRDD<HoodieRecord<HoodieRecordPayload>> hoodieRecords = buildHoodieRecordsForImport(jsc, schemaStr);
|
||||
// Get instant time.
|
||||
String instantTime = client.startCommit();
|
||||
JavaRDD<WriteStatus> writeResponse = load(client, instantTime, hoodieRecords);
|
||||
return UtilHelpers.handleErrors(jsc, instantTime, writeResponse);
|
||||
} catch (Throwable t) {
|
||||
logger.error("Error occurred.", t);
|
||||
}
|
||||
return -1;
|
||||
}
|
||||
|
||||
protected JavaRDD<HoodieRecord<HoodieRecordPayload>> buildHoodieRecordsForImport(
|
||||
JavaSparkContext jsc, String schemaStr) throws IOException {
|
||||
Job job = Job.getInstance(jsc.hadoopConfiguration());
|
||||
// Allow recursive directories to be found
|
||||
job.getConfiguration().set(FileInputFormat.INPUT_DIR_RECURSIVE, "true");
|
||||
// To parallelize reading file status.
|
||||
job.getConfiguration().set(FileInputFormat.LIST_STATUS_NUM_THREADS, "1024");
|
||||
AvroReadSupport
|
||||
.setAvroReadSchema(jsc.hadoopConfiguration(), (new Schema.Parser().parse(schemaStr)));
|
||||
ParquetInputFormat.setReadSupportClass(job, (AvroReadSupport.class));
|
||||
|
||||
return jsc.newAPIHadoopFile(cfg.srcPath,
|
||||
ParquetInputFormat.class, Void.class, GenericRecord.class, job.getConfiguration())
|
||||
// To reduce large number of
|
||||
// tasks.
|
||||
.coalesce(16 * cfg.parallelism)
|
||||
.map(entry -> {
|
||||
GenericRecord genericRecord
|
||||
= ((Tuple2<Void, GenericRecord>) entry)._2();
|
||||
Object partitionField =
|
||||
genericRecord.get(cfg.partitionKey);
|
||||
if (partitionField == null) {
|
||||
throw new HoodieIOException(
|
||||
"partition key is missing. :"
|
||||
+ cfg.partitionKey);
|
||||
}
|
||||
Object rowField = genericRecord.get(cfg.rowKey);
|
||||
if (rowField == null) {
|
||||
throw new HoodieIOException(
|
||||
"row field is missing. :" + cfg.rowKey);
|
||||
}
|
||||
String partitionPath = partitionField.toString();
|
||||
logger.info("Row Key : " + rowField + ", Partition Path is (" + partitionPath + ")");
|
||||
if (partitionField instanceof Number) {
|
||||
try {
|
||||
long ts = (long) (Double.parseDouble(partitionField.toString()) * 1000L);
|
||||
partitionPath =
|
||||
PARTITION_FORMATTER.format(new Date(ts));
|
||||
} catch (NumberFormatException nfe) {
|
||||
logger.warn("Unable to parse date from partition field. Assuming partition as (" + partitionField + ")");
|
||||
}
|
||||
}
|
||||
return new HoodieRecord<>(
|
||||
new HoodieKey(
|
||||
(String) rowField, partitionPath),
|
||||
new HoodieJsonPayload(
|
||||
genericRecord.toString()));
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Imports records to Hoodie dataset
|
||||
*
|
||||
* @param client Hoodie Client
|
||||
* @param instantTime Instant Time
|
||||
* @param hoodieRecords Hoodie Records
|
||||
* @param <T> Type
|
||||
*/
|
||||
protected <T extends HoodieRecordPayload> JavaRDD<WriteStatus> load(HoodieWriteClient client,
|
||||
String instantTime, JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
if (cfg.command.toLowerCase().equals("insert")) {
|
||||
return client.insert(hoodieRecords, instantTime);
|
||||
}
|
||||
return client.upsert(hoodieRecords, instantTime);
|
||||
}
|
||||
|
||||
public static class FormatValidator implements IValueValidator<String> {
|
||||
|
||||
List<String> validFormats = Arrays.asList("parquet");
|
||||
|
||||
@Override
|
||||
public void validate(String name, String value) throws ParameterException {
|
||||
if (value == null || !validFormats.contains(value)) {
|
||||
throw new ParameterException(String.format(
|
||||
"Invalid format type: value:%s: supported formats:%s", value, validFormats));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--command", "-c"},
|
||||
description = "Write command Valid values are insert(default)/upsert",
|
||||
required = false)
|
||||
public String command = "INSERT";
|
||||
@Parameter(names = {"--src-path",
|
||||
"-sp"}, description = "Base path for the input dataset", required = true)
|
||||
public String srcPath = null;
|
||||
@Parameter(names = {"--target-path",
|
||||
"-tp"}, description = "Base path for the target hoodie dataset", required = true)
|
||||
public String targetPath = null;
|
||||
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
|
||||
public String tableName = null;
|
||||
@Parameter(names = {"--table-type", "-tt"}, description = "Table type", required = true)
|
||||
public String tableType = null;
|
||||
@Parameter(names = {"--row-key-field",
|
||||
"-rk"}, description = "Row key field name", required = true)
|
||||
public String rowKey = null;
|
||||
@Parameter(names = {"--partition-key-field",
|
||||
"-pk"}, description = "Partition key field name", required = true)
|
||||
public String partitionKey = null;
|
||||
@Parameter(names = {"--parallelism",
|
||||
"-pl"}, description = "Parallelism for hoodie insert", required = true)
|
||||
public int parallelism = 1;
|
||||
@Parameter(names = {"--schema-file",
|
||||
"-sf"}, description = "path for Avro schema file", required = true)
|
||||
public String schemaFile = null;
|
||||
@Parameter(names = {"--format",
|
||||
"-f"}, description = "Format for the input data.", required = false, validateValueWith =
|
||||
FormatValidator.class)
|
||||
public String format = null;
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory",
|
||||
"-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false)
|
||||
public int retry = 0;
|
||||
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
|
||||
+ "hoodie client for importing")
|
||||
public String propsFilePath = null;
|
||||
@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
|
||||
+ "(using the CLI parameter \"--propsFilePath\") can also be passed command line using this parameter")
|
||||
public List<String> configs = new ArrayList<>();
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
}
|
||||
@@ -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.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.io.File;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.sql.Connection;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.List;
|
||||
import java.util.Scanner;
|
||||
import java.util.stream.Collectors;
|
||||
import javax.sql.DataSource;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.utilities.exception.HoodieIncrementalPullException;
|
||||
import org.apache.hudi.utilities.exception.HoodieIncrementalPullSQLException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.stringtemplate.v4.ST;
|
||||
|
||||
/**
|
||||
* Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as
|
||||
* another hive temporary table.
|
||||
* <p>
|
||||
* Current Limitations:
|
||||
* <p>
|
||||
* - Only the source table can be incrementally pulled (usually the largest table) - The
|
||||
* incrementally pulled table can't be referenced more than once.
|
||||
*/
|
||||
public class HiveIncrementalPuller {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HiveIncrementalPuller.class);
|
||||
private static String driverName = "org.apache.hive.jdbc.HiveDriver";
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--hiveUrl"})
|
||||
public String hiveJDBCUrl = "jdbc:hive2://localhost:10014/;transportMode=http;httpPath=hs2";
|
||||
@Parameter(names = {"--hiveUser"})
|
||||
public String hiveUsername = "hive";
|
||||
@Parameter(names = {"--hivePass"})
|
||||
public String hivePassword = "";
|
||||
@Parameter(names = {"--queue"})
|
||||
public String yarnQueueName = "hadoop-queue";
|
||||
@Parameter(names = {"--tmp"})
|
||||
public String hoodieTmpDir = "/app/hoodie/intermediate";
|
||||
@Parameter(names = {"--extractSQLFile"}, required = true)
|
||||
public String incrementalSQLFile;
|
||||
@Parameter(names = {"--sourceDb"}, required = true)
|
||||
public String sourceDb;
|
||||
@Parameter(names = {"--sourceTable"}, required = true)
|
||||
public String sourceTable;
|
||||
@Parameter(names = {"--targetDb"})
|
||||
public String targetDb;
|
||||
@Parameter(names = {"--targetTable"}, required = true)
|
||||
public String targetTable;
|
||||
@Parameter(names = {"--tmpdb"})
|
||||
public String tmpDb = "tmp";
|
||||
@Parameter(names = {"--fromCommitTime"})
|
||||
public String fromCommitTime;
|
||||
@Parameter(names = {"--maxCommits"})
|
||||
public int maxCommits = 3;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
static {
|
||||
try {
|
||||
Class.forName(driverName);
|
||||
} catch (ClassNotFoundException e) {
|
||||
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
|
||||
}
|
||||
}
|
||||
|
||||
private Connection connection;
|
||||
protected final Config config;
|
||||
private final ST incrementalPullSQLtemplate;
|
||||
|
||||
public HiveIncrementalPuller(Config config) throws IOException {
|
||||
this.config = config;
|
||||
validateConfig(config);
|
||||
String templateContent = IOUtils.toString(
|
||||
this.getClass().getResourceAsStream("IncrementalPull.sqltemplate"));
|
||||
incrementalPullSQLtemplate = new ST(templateContent);
|
||||
}
|
||||
|
||||
private void validateConfig(Config config) {
|
||||
if (config.maxCommits == -1) {
|
||||
config.maxCommits = Integer.MAX_VALUE;
|
||||
}
|
||||
}
|
||||
|
||||
public void saveDelta() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
FileSystem fs = FileSystem.get(conf);
|
||||
Statement stmt = null;
|
||||
try {
|
||||
if (config.fromCommitTime == null) {
|
||||
config.fromCommitTime = inferCommitTime(fs);
|
||||
log.info("FromCommitTime inferred as " + config.fromCommitTime);
|
||||
}
|
||||
|
||||
log.info("FromCommitTime - " + config.fromCommitTime);
|
||||
String sourceTableLocation = getTableLocation(config.sourceDb, config.sourceTable);
|
||||
String lastCommitTime = getLastCommitTimePulled(fs, sourceTableLocation);
|
||||
if (lastCommitTime == null) {
|
||||
log.info("Nothing to pull. However we will continue to create a empty table");
|
||||
lastCommitTime = config.fromCommitTime;
|
||||
}
|
||||
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
// drop the temp table if exists
|
||||
String tempDbTable = config.tmpDb + "." + config.targetTable + "__" + config.sourceTable;
|
||||
String tempDbTablePath =
|
||||
config.hoodieTmpDir + "/" + config.targetTable + "__" + config.sourceTable + "/"
|
||||
+ lastCommitTime;
|
||||
executeStatement("drop table " + tempDbTable, stmt);
|
||||
deleteHDFSPath(fs, tempDbTablePath);
|
||||
if (!ensureTempPathExists(fs, lastCommitTime)) {
|
||||
throw new IllegalStateException(
|
||||
"Could not create target path at " + new Path(config.hoodieTmpDir,
|
||||
config.targetTable + "/" + lastCommitTime));
|
||||
}
|
||||
|
||||
initHiveBeelineProperties(stmt);
|
||||
executeIncrementalSQL(tempDbTable, tempDbTablePath, stmt);
|
||||
log.info("Finished HoodieReader execution");
|
||||
} catch (SQLException e) {
|
||||
log.error("Exception when executing SQL", e);
|
||||
throw new IOException("Could not scan " + config.sourceTable + " incrementally", e);
|
||||
} finally {
|
||||
try {
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
log.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void executeIncrementalSQL(String tempDbTable, String tempDbTablePath, Statement stmt)
|
||||
throws FileNotFoundException, SQLException {
|
||||
incrementalPullSQLtemplate.add("tempDbTable", tempDbTable);
|
||||
incrementalPullSQLtemplate.add("tempDbTablePath", tempDbTablePath);
|
||||
|
||||
String storedAsClause = getStoredAsClause();
|
||||
|
||||
incrementalPullSQLtemplate.add("storedAsClause", storedAsClause);
|
||||
String incrementalSQL = new Scanner(new File(config.incrementalSQLFile)).useDelimiter("\\Z")
|
||||
.next();
|
||||
if (!incrementalSQL.contains(config.sourceDb + "." + config.sourceTable)) {
|
||||
log.info("Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable
|
||||
+ ", which means its pulling from a different table. Fencing this from "
|
||||
+ "happening.");
|
||||
throw new HoodieIncrementalPullSQLException(
|
||||
"Incremental SQL does not have " + config.sourceDb + "." + config.sourceTable);
|
||||
}
|
||||
if (!incrementalSQL.contains("`_hoodie_commit_time` > '%targetBasePath'")) {
|
||||
log.info("Incremental SQL : " + incrementalSQL
|
||||
+ " does not contain `_hoodie_commit_time` > '%targetBasePath'. Please add "
|
||||
+ "this clause for incremental to work properly.");
|
||||
throw new HoodieIncrementalPullSQLException(
|
||||
"Incremental SQL does not have clause `_hoodie_commit_time` > '%targetBasePath', which "
|
||||
+ "means its not pulling incrementally");
|
||||
}
|
||||
|
||||
incrementalPullSQLtemplate
|
||||
.add("incrementalSQL", String.format(incrementalSQL, config.fromCommitTime));
|
||||
String sql = incrementalPullSQLtemplate.render();
|
||||
// Check if the SQL is pulling from the right database
|
||||
executeStatement(sql, stmt);
|
||||
}
|
||||
|
||||
private String getStoredAsClause() {
|
||||
return "STORED AS AVRO";
|
||||
}
|
||||
|
||||
private void initHiveBeelineProperties(Statement stmt) throws SQLException {
|
||||
log.info("Setting up Hive JDBC Session with properties");
|
||||
// set the queue
|
||||
executeStatement("set mapred.job.queue.name=" + config.yarnQueueName, stmt);
|
||||
// Set the inputformat to HoodieCombineHiveInputFormat
|
||||
executeStatement(
|
||||
"set hive.input.format=org.apache.hudi.hadoop.hive.HoodieCombineHiveInputFormat", stmt);
|
||||
// Allow queries without partition predicate
|
||||
executeStatement("set hive.strict.checks.large.query=false", stmt);
|
||||
// Dont gather stats for the table created
|
||||
executeStatement("set hive.stats.autogather=false", stmt);
|
||||
// Set the hoodie modie
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.mode=INCREMENTAL", stmt);
|
||||
// Set the from commit time
|
||||
executeStatement(
|
||||
"set hoodie." + config.sourceTable + ".consume.start.timestamp=" + config.fromCommitTime,
|
||||
stmt);
|
||||
// Set number of commits to pull
|
||||
executeStatement("set hoodie." + config.sourceTable + ".consume.max.commits=" + String.valueOf(
|
||||
config.maxCommits), stmt);
|
||||
}
|
||||
|
||||
private boolean deleteHDFSPath(FileSystem fs, String path) throws IOException {
|
||||
log.info("Deleting path " + path);
|
||||
return fs.delete(new Path(path), true);
|
||||
}
|
||||
|
||||
private void executeStatement(String sql, Statement stmt) throws SQLException {
|
||||
log.info("Executing: " + sql);
|
||||
stmt.execute(sql);
|
||||
}
|
||||
|
||||
private String inferCommitTime(FileSystem fs) throws SQLException, IOException {
|
||||
log.info(
|
||||
"FromCommitTime not specified. Trying to infer it from Hoodie dataset " + config.targetDb
|
||||
+ "." + config.targetTable);
|
||||
String targetDataLocation = getTableLocation(config.targetDb, config.targetTable);
|
||||
return scanForCommitTime(fs, targetDataLocation);
|
||||
}
|
||||
|
||||
private String getTableLocation(String db, String table) throws SQLException {
|
||||
ResultSet resultSet = null;
|
||||
Statement stmt = null;
|
||||
try {
|
||||
Connection conn = getConnection();
|
||||
stmt = conn.createStatement();
|
||||
resultSet = stmt.executeQuery("describe formatted `" + db + "." + table + "`");
|
||||
while (resultSet.next()) {
|
||||
if (resultSet.getString(1).trim().equals("Location:")) {
|
||||
log.info(
|
||||
"Inferred table location for " + db + "." + table + " as " + resultSet.getString(2));
|
||||
return resultSet.getString(2);
|
||||
}
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieIncrementalPullException(
|
||||
"Failed to get data location for table " + db + "." + table, e);
|
||||
} finally {
|
||||
try {
|
||||
if (stmt != null) {
|
||||
stmt.close();
|
||||
}
|
||||
if (resultSet != null) {
|
||||
resultSet.close();
|
||||
}
|
||||
} catch (SQLException e) {
|
||||
log.error("Could not close the resultset opened ", e);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IOException {
|
||||
if (targetDataPath == null) {
|
||||
throw new IllegalArgumentException(
|
||||
"Please specify either --fromCommitTime or --targetDataPath");
|
||||
}
|
||||
if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) {
|
||||
return "0";
|
||||
}
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath);
|
||||
|
||||
Option<HoodieInstant> lastCommit = metadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
if (lastCommit.isPresent()) {
|
||||
return lastCommit.get().getTimestamp();
|
||||
}
|
||||
return "0";
|
||||
}
|
||||
|
||||
private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) throws IOException {
|
||||
Path targetBaseDirPath = new Path(config.hoodieTmpDir,
|
||||
config.targetTable + "__" + config.sourceTable);
|
||||
if (!fs.exists(targetBaseDirPath)) {
|
||||
log.info("Creating " + targetBaseDirPath + " with permission drwxrwxrwx");
|
||||
boolean result = FileSystem.mkdirs(fs, targetBaseDirPath,
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
if (!result) {
|
||||
throw new HoodieException(
|
||||
"Could not create " + targetBaseDirPath + " with the required permissions");
|
||||
}
|
||||
}
|
||||
|
||||
Path targetPath = new Path(targetBaseDirPath, lastCommitTime);
|
||||
if (fs.exists(targetPath)) {
|
||||
boolean result = fs.delete(targetPath, true);
|
||||
if (!result) {
|
||||
throw new HoodieException("Could not delete existing " + targetPath);
|
||||
}
|
||||
}
|
||||
log.info("Creating " + targetPath + " with permission drwxrwxrwx");
|
||||
return FileSystem.mkdirs(fs, targetBaseDirPath,
|
||||
new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
|
||||
}
|
||||
|
||||
private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation)
|
||||
throws IOException {
|
||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), sourceTableLocation);
|
||||
List<String> commitsToSync = metadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants()
|
||||
.findInstantsAfter(config.fromCommitTime, config.maxCommits)
|
||||
.getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if (commitsToSync.isEmpty()) {
|
||||
log.warn("Nothing to sync. All commits in " + config.sourceTable + " are "
|
||||
+ metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
|
||||
.getInstants().collect(Collectors.toList())
|
||||
+ " and from commit time is " + config.fromCommitTime);
|
||||
return null;
|
||||
}
|
||||
log.info("Syncing commits " + commitsToSync);
|
||||
return commitsToSync.get(commitsToSync.size() - 1);
|
||||
}
|
||||
|
||||
private Connection getConnection() throws SQLException {
|
||||
if (connection == null) {
|
||||
DataSource ds = getDatasource();
|
||||
log.info("Getting Hive Connection from Datasource " + ds);
|
||||
this.connection = ds.getConnection();
|
||||
}
|
||||
return connection;
|
||||
}
|
||||
|
||||
private DataSource getDatasource() {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
ds.setUrl(config.hiveJDBCUrl);
|
||||
ds.setUsername(config.hiveUsername);
|
||||
ds.setPassword(config.hivePassword);
|
||||
return ds;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
new HiveIncrementalPuller(cfg).saveDelta();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,115 @@
|
||||
/*
|
||||
* 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.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class HoodieCleaner {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(HoodieCleaner.class);
|
||||
|
||||
/**
|
||||
* Config for Cleaner
|
||||
*/
|
||||
private final Config cfg;
|
||||
|
||||
/**
|
||||
* Filesystem used
|
||||
*/
|
||||
private transient FileSystem fs;
|
||||
|
||||
/**
|
||||
* Spark context
|
||||
*/
|
||||
private transient JavaSparkContext jssc;
|
||||
|
||||
/**
|
||||
* Bag of properties with source, hoodie client, key generator etc.
|
||||
*/
|
||||
private TypedProperties props;
|
||||
|
||||
public HoodieCleaner(Config cfg, JavaSparkContext jssc) throws IOException {
|
||||
this.cfg = cfg;
|
||||
this.jssc = jssc;
|
||||
this.fs = FSUtils.getFs(cfg.basePath, jssc.hadoopConfiguration());
|
||||
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) :
|
||||
UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
||||
log.info("Creating Cleaner with configs : " + props.toString());
|
||||
}
|
||||
|
||||
public void run() throws Exception {
|
||||
HoodieWriteConfig hoodieCfg = getHoodieClientConfig();
|
||||
HoodieWriteClient client = new HoodieWriteClient<>(jssc, hoodieCfg, false);
|
||||
client.clean();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getHoodieClientConfig() throws Exception {
|
||||
return HoodieWriteConfig.newBuilder().combineInput(true, true).withPath(cfg.basePath)
|
||||
.withAutoCommit(false)
|
||||
.withProps(props).build();
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--target-base-path"}, description = "base path for the hoodie dataset to be cleaner.",
|
||||
required = true)
|
||||
public String basePath;
|
||||
|
||||
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
|
||||
+ "hoodie client for cleaning")
|
||||
public String propsFilePath = null;
|
||||
|
||||
@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
|
||||
+ "(using the CLI parameter \"--propsFilePath\") can also be passed command line using this parameter")
|
||||
public List<String> configs = new ArrayList<>();
|
||||
|
||||
@Parameter(names = {"--spark-master"}, description = "spark master to use.")
|
||||
public String sparkMaster = "local[2]";
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
String dirName = new Path(cfg.basePath).getName();
|
||||
JavaSparkContext jssc = UtilHelpers.buildSparkContext("hoodie-cleaner-" + dirName, cfg.sparkMaster);
|
||||
new HoodieCleaner(cfg, jssc).run();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,176 @@
|
||||
/*
|
||||
* 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.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.CompactionAdminClient;
|
||||
import org.apache.hudi.CompactionAdminClient.RenameOpResult;
|
||||
import org.apache.hudi.CompactionAdminClient.ValidationOpResult;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class HoodieCompactionAdminTool {
|
||||
|
||||
private final Config cfg;
|
||||
|
||||
public HoodieCompactionAdminTool(Config cfg) {
|
||||
this.cfg = cfg;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
HoodieCompactionAdminTool admin = new HoodieCompactionAdminTool(cfg);
|
||||
admin.run(UtilHelpers.buildSparkContext("admin-compactor", cfg.sparkMaster, cfg.sparkMemory));
|
||||
}
|
||||
|
||||
/**
|
||||
* Executes one of compaction admin operations
|
||||
*/
|
||||
public void run(JavaSparkContext jsc) throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
|
||||
final CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath);
|
||||
try {
|
||||
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
|
||||
if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
|
||||
throw new IllegalStateException("Output File Path already exists");
|
||||
}
|
||||
switch (cfg.operation) {
|
||||
case VALIDATE:
|
||||
List<ValidationOpResult> res =
|
||||
admin.validateCompactionPlan(metaClient, cfg.compactionInstantTime, cfg.parallelism);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Validation Operation :", res);
|
||||
}
|
||||
serializeOperationResult(fs, res);
|
||||
break;
|
||||
case UNSCHEDULE_FILE:
|
||||
List<RenameOpResult> r =
|
||||
admin.unscheduleCompactionFileId(new HoodieFileGroupId(cfg.partitionPath, cfg.fileId),
|
||||
cfg.skipValidation, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
System.out.println(r);
|
||||
}
|
||||
serializeOperationResult(fs, r);
|
||||
break;
|
||||
case UNSCHEDULE_PLAN:
|
||||
List<RenameOpResult> r2 =
|
||||
admin
|
||||
.unscheduleCompactionPlan(cfg.compactionInstantTime, cfg.skipValidation, cfg.parallelism, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Unscheduling Compaction Plan :", r2);
|
||||
}
|
||||
serializeOperationResult(fs, r2);
|
||||
break;
|
||||
case REPAIR:
|
||||
List<RenameOpResult> r3 =
|
||||
admin.repairCompaction(cfg.compactionInstantTime, cfg.parallelism, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Repair Operation :", r3);
|
||||
}
|
||||
serializeOperationResult(fs, r3);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Not yet implemented !!");
|
||||
}
|
||||
} finally {
|
||||
admin.close();
|
||||
}
|
||||
}
|
||||
|
||||
private <T> void serializeOperationResult(FileSystem fs, T result) throws Exception {
|
||||
if ((cfg.outputPath != null) && (result != null)) {
|
||||
Path outputPath = new Path(cfg.outputPath);
|
||||
FSDataOutputStream fsout = fs.create(outputPath, true);
|
||||
ObjectOutputStream out = new ObjectOutputStream(fsout);
|
||||
out.writeObject(result);
|
||||
out.close();
|
||||
fsout.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Print Operation Result
|
||||
*
|
||||
* @param initialLine Initial Line
|
||||
* @param result Result
|
||||
*/
|
||||
private <T> void printOperationResult(String initialLine, List<T> result) {
|
||||
System.out.println(initialLine);
|
||||
for (T r : result) {
|
||||
System.out.print(r);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Operation Types
|
||||
*/
|
||||
public enum Operation {
|
||||
VALIDATE,
|
||||
UNSCHEDULE_PLAN,
|
||||
UNSCHEDULE_FILE,
|
||||
REPAIR
|
||||
}
|
||||
|
||||
/**
|
||||
* Admin Configuration Options
|
||||
*/
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--operation", "-op"}, description = "Operation", required = true)
|
||||
public Operation operation = Operation.VALIDATE;
|
||||
@Parameter(names = {"--base-path", "-bp"}, description = "Base path for the dataset", required = true)
|
||||
public String basePath = null;
|
||||
@Parameter(names = {"--instant-time", "-in"}, description = "Compaction Instant time", required = false)
|
||||
public String compactionInstantTime = null;
|
||||
@Parameter(names = {"--partition-path", "-pp"}, description = "Partition Path", required = false)
|
||||
public String partitionPath = null;
|
||||
@Parameter(names = {"--file-id", "-id"}, description = "File Id", required = false)
|
||||
public String fileId = null;
|
||||
@Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false)
|
||||
public int parallelism = 3;
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = true)
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--dry-run", "-dr"}, description = "Dry Run Mode", required = false)
|
||||
public boolean dryRun = false;
|
||||
@Parameter(names = {"--skip-validation", "-sv"}, description = "Skip Validation", required = false)
|
||||
public boolean skipValidation = false;
|
||||
@Parameter(names = {"--output-path", "-ot"}, description = "Output Path", required = false)
|
||||
public String outputPath = null;
|
||||
@Parameter(names = {"--print-output", "-pt"}, description = "Print Output", required = false)
|
||||
public boolean printOutput = true;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,138 @@
|
||||
/*
|
||||
* 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.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
|
||||
public class HoodieCompactor {
|
||||
|
||||
private static volatile Logger logger = LogManager.getLogger(HoodieCompactor.class);
|
||||
private final Config cfg;
|
||||
private transient FileSystem fs;
|
||||
private TypedProperties props;
|
||||
|
||||
public HoodieCompactor(Config cfg) {
|
||||
this.cfg = cfg;
|
||||
this.props = cfg.propsFilePath == null ? UtilHelpers.buildProperties(cfg.configs) :
|
||||
UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
@Parameter(names = {"--base-path",
|
||||
"-sp"}, description = "Base path for the dataset", required = true)
|
||||
public String basePath = null;
|
||||
@Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true)
|
||||
public String tableName = null;
|
||||
@Parameter(names = {"--instant-time",
|
||||
"-sp"}, description = "Compaction Instant time", required = true)
|
||||
public String compactionInstantTime = null;
|
||||
@Parameter(names = {"--parallelism",
|
||||
"-pl"}, description = "Parallelism for hoodie insert", required = true)
|
||||
public int parallelism = 1;
|
||||
@Parameter(names = {"--schema-file",
|
||||
"-sf"}, description = "path for Avro schema file", required = true)
|
||||
public String schemaFile = null;
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory",
|
||||
"-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false)
|
||||
public int retry = 0;
|
||||
@Parameter(names = {"--schedule", "-sc"}, description = "Schedule compaction", required = false)
|
||||
public Boolean runSchedule = false;
|
||||
@Parameter(names = {"--strategy", "-st"}, description = "Stratgey Class", required = false)
|
||||
public String strategyClassName = null;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
|
||||
+ "hoodie client for compacting")
|
||||
public String propsFilePath = null;
|
||||
|
||||
@Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file "
|
||||
+ "(using the CLI parameter \"--propsFilePath\") can also be passed command line using this parameter")
|
||||
public List<String> configs = new ArrayList<>();
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
HoodieCompactor compactor = new HoodieCompactor(cfg);
|
||||
compactor.compact(UtilHelpers.buildSparkContext("compactor-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory),
|
||||
cfg.retry);
|
||||
}
|
||||
|
||||
public int compact(JavaSparkContext jsc, int retry) {
|
||||
this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
|
||||
int ret = -1;
|
||||
try {
|
||||
do {
|
||||
if (cfg.runSchedule) {
|
||||
if (null == cfg.strategyClassName) {
|
||||
throw new IllegalArgumentException("Missing Strategy class name for running compaction");
|
||||
}
|
||||
ret = doSchedule(jsc);
|
||||
} else {
|
||||
ret = doCompact(jsc);
|
||||
}
|
||||
} while (ret != 0 && retry-- > 0);
|
||||
} catch (Throwable t) {
|
||||
logger.error(t);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private int doCompact(JavaSparkContext jsc) throws Exception {
|
||||
//Get schema.
|
||||
String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile);
|
||||
HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism,
|
||||
Option.empty(), props);
|
||||
JavaRDD<WriteStatus> writeResponse = client.compact(cfg.compactionInstantTime);
|
||||
return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse);
|
||||
}
|
||||
|
||||
private int doSchedule(JavaSparkContext jsc) throws Exception {
|
||||
//Get schema.
|
||||
HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism,
|
||||
Option.of(cfg.strategyClassName), props);
|
||||
client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Option.empty());
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
@@ -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.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie snapshot copy job which copies latest files from all partitions to another place, for
|
||||
* snapshot backup.
|
||||
*/
|
||||
public class HoodieSnapshotCopier implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieSnapshotCopier.class);
|
||||
|
||||
static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--base-path",
|
||||
"-bp"}, description = "Hoodie table base path", required = true)
|
||||
String basePath = null;
|
||||
|
||||
@Parameter(names = {"--output-path",
|
||||
"-op"}, description = "The snapshot output path", required = true)
|
||||
String outputPath = null;
|
||||
|
||||
@Parameter(names = {"--date-partitioned",
|
||||
"-dp"}, description = "Can we assume date partitioning?")
|
||||
boolean shouldAssumeDatePartitioning = false;
|
||||
}
|
||||
|
||||
public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDir,
|
||||
final boolean shouldAssumeDatePartitioning) throws IOException {
|
||||
FileSystem fs = FSUtils.getFs(baseDir, jsc.hadoopConfiguration());
|
||||
final SerializableConfiguration serConf = new SerializableConfiguration(
|
||||
jsc.hadoopConfiguration());
|
||||
final HoodieTableMetaClient tableMetadata = new HoodieTableMetaClient(fs.getConf(), baseDir);
|
||||
final ReadOptimizedView fsView = new HoodieTableFileSystemView(
|
||||
tableMetadata,
|
||||
tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants());
|
||||
// Get the latest commit
|
||||
Option<HoodieInstant> latestCommit = tableMetadata.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
if (!latestCommit.isPresent()) {
|
||||
logger.warn("No commits present. Nothing to snapshot");
|
||||
return;
|
||||
}
|
||||
final String latestCommitTimestamp = latestCommit.get().getTimestamp();
|
||||
logger.info(String.format(
|
||||
"Starting to snapshot latest version files which are also no-late-than %s.",
|
||||
latestCommitTimestamp));
|
||||
|
||||
List<String> partitions = FSUtils
|
||||
.getAllPartitionPaths(fs, baseDir, shouldAssumeDatePartitioning);
|
||||
if (partitions.size() > 0) {
|
||||
logger.info(String.format("The job needs to copy %d partitions.", partitions.size()));
|
||||
|
||||
// Make sure the output directory is empty
|
||||
Path outputPath = new Path(outputDir);
|
||||
if (fs.exists(outputPath)) {
|
||||
logger.warn(
|
||||
String.format("The output path %s targetBasePath already exists, deleting", outputPath));
|
||||
fs.delete(new Path(outputDir), true);
|
||||
}
|
||||
|
||||
jsc.parallelize(partitions, partitions.size()).flatMap(partition -> {
|
||||
// Only take latest version files <= latestCommit.
|
||||
FileSystem fs1 = FSUtils.getFs(baseDir, serConf.get());
|
||||
List<Tuple2<String, String>> filePaths = new ArrayList<>();
|
||||
Stream<HoodieDataFile> dataFiles = fsView.getLatestDataFilesBeforeOrOn(partition,
|
||||
latestCommitTimestamp);
|
||||
dataFiles.forEach(
|
||||
hoodieDataFile -> filePaths.add(new Tuple2<>(partition, hoodieDataFile.getPath())));
|
||||
|
||||
// also need to copy over partition metadata
|
||||
Path partitionMetaFile = new Path(new Path(baseDir, partition),
|
||||
HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE);
|
||||
if (fs1.exists(partitionMetaFile)) {
|
||||
filePaths.add(new Tuple2<>(partition, partitionMetaFile.toString()));
|
||||
}
|
||||
|
||||
return filePaths.iterator();
|
||||
}).foreach(tuple -> {
|
||||
String partition = tuple._1();
|
||||
Path sourceFilePath = new Path(tuple._2());
|
||||
Path toPartitionPath = new Path(outputDir, partition);
|
||||
FileSystem ifs = FSUtils.getFs(baseDir, serConf.get());
|
||||
|
||||
if (!ifs.exists(toPartitionPath)) {
|
||||
ifs.mkdirs(toPartitionPath);
|
||||
}
|
||||
FileUtil.copy(ifs, sourceFilePath, ifs, new Path(toPartitionPath, sourceFilePath.getName()),
|
||||
false, ifs.getConf());
|
||||
});
|
||||
|
||||
// Also copy the .commit files
|
||||
logger.info(
|
||||
String.format("Copying .commit files which are no-late-than %s.", latestCommitTimestamp));
|
||||
FileStatus[] commitFilesToCopy = fs.listStatus(
|
||||
new Path(baseDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME), (commitFilePath) -> {
|
||||
if (commitFilePath.getName().equals(HoodieTableConfig.HOODIE_PROPERTIES_FILE)) {
|
||||
return true;
|
||||
} else {
|
||||
String commitTime = FSUtils.getCommitFromCommitFile(commitFilePath.getName());
|
||||
return HoodieTimeline.compareTimestamps(commitTime, latestCommitTimestamp,
|
||||
HoodieTimeline.LESSER_OR_EQUAL);
|
||||
}
|
||||
});
|
||||
for (FileStatus commitStatus : commitFilesToCopy) {
|
||||
Path targetFilePath = new Path(
|
||||
outputDir + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + commitStatus.getPath()
|
||||
.getName());
|
||||
if (!fs.exists(targetFilePath.getParent())) {
|
||||
fs.mkdirs(targetFilePath.getParent());
|
||||
}
|
||||
if (fs.exists(targetFilePath)) {
|
||||
logger.error(String.format(
|
||||
"The target output commit file (%s targetBasePath) already exists.", targetFilePath));
|
||||
}
|
||||
FileUtil.copy(fs, commitStatus.getPath(), fs, targetFilePath, false, fs.getConf());
|
||||
}
|
||||
} else {
|
||||
logger.info("The job has 0 partition to copy.");
|
||||
}
|
||||
|
||||
// Create the _SUCCESS tag
|
||||
Path successTagPath = new Path(outputDir + "/_SUCCESS");
|
||||
if (!fs.exists(successTagPath)) {
|
||||
logger.info(String.format(
|
||||
"Creating _SUCCESS under targetBasePath: $s", outputDir));
|
||||
fs.createNewFile(successTagPath);
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
// Take input configs
|
||||
final Config cfg = new Config();
|
||||
new JCommander(cfg, args);
|
||||
logger.info(String.format("Snapshot hoodie table from %s targetBasePath to %stargetBasePath",
|
||||
cfg.basePath, cfg.outputPath));
|
||||
|
||||
// Create a spark job to do the snapshot copy
|
||||
SparkConf sparkConf = new SparkConf().setAppName("Hoodie-snapshot-copier");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
JavaSparkContext jsc = new JavaSparkContext(sparkConf);
|
||||
logger.info("Initializing spark job.");
|
||||
|
||||
// Copy
|
||||
HoodieSnapshotCopier copier = new HoodieSnapshotCopier();
|
||||
copier.snapshot(jsc, cfg.basePath, cfg.outputPath, cfg.shouldAssumeDatePartitioning);
|
||||
|
||||
// Stop the job
|
||||
jsc.stop();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,129 @@
|
||||
/*
|
||||
* 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.utilities;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.google.common.base.Preconditions;
|
||||
import io.javalin.Javalin;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.InputStreamReader;
|
||||
import java.io.Serializable;
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.http.HttpResponse;
|
||||
import org.apache.http.client.methods.HttpGet;
|
||||
import org.apache.http.impl.client.CloseableHttpClient;
|
||||
import org.apache.http.impl.client.HttpClientBuilder;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class HoodieWithTimelineServer implements Serializable {
|
||||
|
||||
private static volatile Logger logger = LogManager.getLogger(HoodieCompactor.class);
|
||||
private final Config cfg;
|
||||
private transient FileSystem fs;
|
||||
|
||||
private transient Javalin app = null;
|
||||
|
||||
public HoodieWithTimelineServer(Config cfg) {
|
||||
this.cfg = cfg;
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
|
||||
public String sparkMaster = null;
|
||||
@Parameter(names = {"--spark-memory",
|
||||
"-sm"}, description = "spark memory to use", required = true)
|
||||
public String sparkMemory = null;
|
||||
@Parameter(names = {"--num-partitions", "-n"}, description = "Num Partitions", required = false)
|
||||
public Integer numPartitions = 100;
|
||||
@Parameter(names = {"--server-port", "-p"}, description = " Server Port", required = false)
|
||||
public Integer serverPort = 26754;
|
||||
@Parameter(names = {"--delay-secs", "-d"}, description = "Delay(sec) before client connects", required = false)
|
||||
public Integer delaySecs = 30;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public void startService() {
|
||||
app = Javalin.create().start(cfg.serverPort);
|
||||
app.get("/", ctx -> ctx.result("Hello World"));
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
HoodieWithTimelineServer service = new HoodieWithTimelineServer(cfg);
|
||||
service.run(UtilHelpers.buildSparkContext("client-server-hoodie", cfg.sparkMaster, cfg.sparkMemory));
|
||||
}
|
||||
|
||||
public void run(JavaSparkContext jsc) throws UnknownHostException {
|
||||
startService();
|
||||
final String driverHost = InetAddress.getLocalHost().getHostAddress();
|
||||
System.out.println("Driver Hostname is :" + driverHost);
|
||||
List<String> messages = new ArrayList<>();
|
||||
IntStream.range(0, cfg.numPartitions).forEach(i -> messages.add("Hello World"));
|
||||
List<String> gotMessages =
|
||||
jsc.parallelize(messages).map(msg -> sendRequest(driverHost, cfg.serverPort)).collect();
|
||||
System.out.println("Got Messages :" + gotMessages);
|
||||
Preconditions.checkArgument(gotMessages.equals(messages), "Got expected reply from Server");
|
||||
}
|
||||
|
||||
public String sendRequest(String driverHost, int port) throws RuntimeException {
|
||||
String url = String.format("http://%s:%d/", driverHost, port);
|
||||
try {
|
||||
|
||||
System.out.println("Sleeping for " + cfg.delaySecs + " secs ");
|
||||
Thread.sleep(cfg.delaySecs * 1000);
|
||||
System.out.println("Woke up after sleeping for " + cfg.delaySecs + " secs ");
|
||||
|
||||
CloseableHttpClient client = HttpClientBuilder.create().build();
|
||||
HttpGet request = new HttpGet(url);
|
||||
|
||||
HttpResponse response = client.execute(request);
|
||||
|
||||
System.out.println("Response Code from(" + url + ") : " + response.getStatusLine().getStatusCode());
|
||||
|
||||
BufferedReader rd = new BufferedReader(
|
||||
new InputStreamReader(response.getEntity().getContent()));
|
||||
|
||||
StringBuffer result = new StringBuffer();
|
||||
String line = "";
|
||||
while ((line = rd.readLine()) != null) {
|
||||
result.append(line);
|
||||
}
|
||||
System.out.println("Got result (" + result + ")");
|
||||
return result.toString();
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.utilities;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.StringReader;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.util.DFSPropertiesConfiguration;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.Source;
|
||||
import org.apache.hudi.utilities.transform.Transformer;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Accumulator;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
/**
|
||||
* Bunch of helper methods
|
||||
*/
|
||||
public class UtilHelpers {
|
||||
private static Logger logger = LogManager.getLogger(UtilHelpers.class);
|
||||
|
||||
public static Source createSource(String sourceClass, TypedProperties cfg,
|
||||
JavaSparkContext jssc, SparkSession sparkSession, SchemaProvider schemaProvider)
|
||||
throws IOException {
|
||||
try {
|
||||
return (Source) ReflectionUtils.loadClass(sourceClass,
|
||||
new Class<?>[]{TypedProperties.class, JavaSparkContext.class, SparkSession.class, SchemaProvider.class},
|
||||
cfg, jssc, sparkSession, schemaProvider);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load source class " + sourceClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static SchemaProvider createSchemaProvider(String schemaProviderClass,
|
||||
TypedProperties cfg, JavaSparkContext jssc) throws IOException {
|
||||
try {
|
||||
return schemaProviderClass == null ? null :
|
||||
(SchemaProvider) ReflectionUtils.loadClass(schemaProviderClass, cfg, jssc);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load schema provider class " + schemaProviderClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static Transformer createTransformer(String transformerClass) throws IOException {
|
||||
try {
|
||||
return transformerClass == null ? null : (Transformer) ReflectionUtils.loadClass(transformerClass);
|
||||
} catch (Throwable e) {
|
||||
throw new IOException("Could not load transformer class " + transformerClass, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
*/
|
||||
public static DFSPropertiesConfiguration readConfig(FileSystem fs, Path cfgPath, List<String> overriddenProps) {
|
||||
try {
|
||||
DFSPropertiesConfiguration conf = new DFSPropertiesConfiguration(fs, cfgPath);
|
||||
if (!overriddenProps.isEmpty()) {
|
||||
logger.info("Adding overridden properties to file properties.");
|
||||
conf.addProperties(new BufferedReader(new StringReader(String.join("\n", overriddenProps))));
|
||||
}
|
||||
return conf;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Unable to read props file at :" + cfgPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
public static TypedProperties buildProperties(List<String> props) {
|
||||
TypedProperties properties = new TypedProperties();
|
||||
props.stream().forEach(x -> {
|
||||
String[] kv = x.split("=");
|
||||
Preconditions.checkArgument(kv.length == 2);
|
||||
properties.setProperty(kv[0], kv[1]);
|
||||
});
|
||||
return properties;
|
||||
}
|
||||
|
||||
/**
|
||||
* Parse Schema from file
|
||||
*
|
||||
* @param fs File System
|
||||
* @param schemaFile Schema File
|
||||
*/
|
||||
public static String parseSchema(FileSystem fs, String schemaFile) throws Exception {
|
||||
// Read schema file.
|
||||
Path p = new Path(schemaFile);
|
||||
if (!fs.exists(p)) {
|
||||
throw new Exception(String.format("Could not find - %s - schema file.", schemaFile));
|
||||
}
|
||||
long len = fs.getFileStatus(p).getLen();
|
||||
ByteBuffer buf = ByteBuffer.allocate((int) len);
|
||||
try (FSDataInputStream inputStream = fs.open(p)) {
|
||||
inputStream.readFully(0, buf.array(), 0, buf.array().length);
|
||||
}
|
||||
return new String(buf.array());
|
||||
}
|
||||
|
||||
private static SparkConf buildSparkConf(String appName, String defaultMaster) {
|
||||
return buildSparkConf(appName, defaultMaster, new HashMap<>());
|
||||
}
|
||||
|
||||
private static SparkConf buildSparkConf(String appName, String defaultMaster, Map<String, String> additionalConfigs) {
|
||||
final SparkConf sparkConf = new SparkConf().setAppName(appName);
|
||||
String master = sparkConf.get("spark.master", defaultMaster);
|
||||
sparkConf.setMaster(master);
|
||||
if (master.startsWith("yarn")) {
|
||||
sparkConf.set("spark.eventLog.overwrite", "true");
|
||||
sparkConf.set("spark.eventLog.enabled", "true");
|
||||
}
|
||||
sparkConf.setIfMissing("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec",
|
||||
"org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
|
||||
additionalConfigs.entrySet().forEach(e -> sparkConf.set(e.getKey(), e.getValue()));
|
||||
SparkConf newSparkConf = HoodieWriteClient.registerClasses(sparkConf);
|
||||
return newSparkConf;
|
||||
}
|
||||
|
||||
public static JavaSparkContext buildSparkContext(String appName, String defaultMaster, Map<String, String> configs) {
|
||||
return new JavaSparkContext(buildSparkConf(appName, defaultMaster, configs));
|
||||
}
|
||||
|
||||
public static JavaSparkContext buildSparkContext(String appName, String defaultMaster) {
|
||||
return new JavaSparkContext(buildSparkConf(appName, defaultMaster));
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Spark Context for ingestion/compaction
|
||||
* @return
|
||||
*/
|
||||
public static JavaSparkContext buildSparkContext(String appName, String sparkMaster, String sparkMemory) {
|
||||
SparkConf sparkConf = buildSparkConf(appName, sparkMaster);
|
||||
sparkConf.set("spark.executor.memory", sparkMemory);
|
||||
return new JavaSparkContext(sparkConf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hoodie write client
|
||||
*
|
||||
* @param jsc Java Spark Context
|
||||
* @param basePath Base Path
|
||||
* @param schemaStr Schema
|
||||
* @param parallelism Parallelism
|
||||
*/
|
||||
public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath,
|
||||
String schemaStr, int parallelism, Option<String> compactionStrategyClass, TypedProperties properties)
|
||||
throws Exception {
|
||||
HoodieCompactionConfig compactionConfig =
|
||||
compactionStrategyClass.map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false)
|
||||
.withCompactionStrategy(ReflectionUtils.loadClass(strategy))
|
||||
.build()).orElse(HoodieCompactionConfig.newBuilder().withInlineCompaction(false).build());
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withParallelism(parallelism, parallelism).withSchema(schemaStr)
|
||||
.combineInput(true, true)
|
||||
.withCompactionConfig(compactionConfig)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withProps(properties)
|
||||
.build();
|
||||
return new HoodieWriteClient(jsc, config);
|
||||
}
|
||||
|
||||
public static int handleErrors(JavaSparkContext jsc, String instantTime, JavaRDD<WriteStatus> writeResponse) {
|
||||
Accumulator<Integer> errors = jsc.accumulator(0);
|
||||
writeResponse.foreach(writeStatus -> {
|
||||
if (writeStatus.hasErrors()) {
|
||||
errors.add(1);
|
||||
logger.error(String.format("Error processing records :writeStatus:%s",
|
||||
writeStatus.getStat().toString()));
|
||||
}
|
||||
});
|
||||
if (errors.value() == 0) {
|
||||
logger.info(
|
||||
String.format("Dataset imported into hoodie dataset with %s instant time.", instantTime));
|
||||
return 0;
|
||||
}
|
||||
logger.error(String.format("Import failed with %d errors.", errors.value()));
|
||||
return -1;
|
||||
}
|
||||
|
||||
public static TypedProperties readConfig(InputStream in) throws IOException {
|
||||
TypedProperties defaults = new TypedProperties();
|
||||
defaults.load(in);
|
||||
return defaults;
|
||||
}
|
||||
}
|
||||
@@ -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.utilities.adhoc;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import java.io.BufferedReader;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
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.FSUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* This is an one-time use class meant for migrating the configuration for
|
||||
* "hoodie.compaction.payload.class" in .hoodie/hoodie.properties from com.uber.hoodie to
|
||||
* org.apache.hudi
|
||||
* It takes in a file containing base-paths for a set of hudi datasets and does the migration
|
||||
*/
|
||||
public class UpgradePayloadFromUberToApache implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(UpgradePayloadFromUberToApache.class);
|
||||
|
||||
private final Config cfg;
|
||||
|
||||
public UpgradePayloadFromUberToApache(Config cfg) {
|
||||
this.cfg = cfg;
|
||||
}
|
||||
|
||||
public void run() throws IOException {
|
||||
BufferedReader reader = new BufferedReader(new FileReader(cfg.inputPath));
|
||||
String basePath = reader.readLine();
|
||||
while (basePath != null) {
|
||||
basePath = basePath.trim();
|
||||
if (!basePath.startsWith("#")) {
|
||||
logger.info("Performing upgrade for " + basePath);
|
||||
String metaPath = String.format("%s/.hoodie", basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(
|
||||
FSUtils.prepareHadoopConf(new Configuration()), basePath, false);
|
||||
HoodieTableConfig tableConfig = metaClient.getTableConfig();
|
||||
if (tableConfig.getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
Map<String, String> propsMap = tableConfig.getProps();
|
||||
if (propsMap.containsKey(HoodieCompactionConfig.PAYLOAD_CLASS_PROP)) {
|
||||
String payloadClass = propsMap.get(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
|
||||
logger.info("Found payload class=" + payloadClass);
|
||||
if (payloadClass.startsWith("com.uber.hoodie")) {
|
||||
String newPayloadClass = payloadClass.replace("com.uber.hoodie",
|
||||
"org.apache.hudi");
|
||||
logger.info("Replacing payload class (" + payloadClass
|
||||
+ ") with (" + newPayloadClass + ")");
|
||||
Map<String, String> newPropsMap = new HashMap<>(propsMap);
|
||||
newPropsMap.put(HoodieCompactionConfig.PAYLOAD_CLASS_PROP, newPayloadClass);
|
||||
Properties props = new Properties();
|
||||
props.putAll(newPropsMap);
|
||||
tableConfig.createHoodieProperties(metaClient.getFs(), new Path(metaPath), props);
|
||||
logger.info("Finished upgrade for " + basePath);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
logger.info("Skipping as this table is COW table. BasePath=" + basePath);
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--datasets_list_path", "-sp"},
|
||||
description = "Local File containing list of base-paths for which migration needs to be performed",
|
||||
required = true)
|
||||
public String inputPath = null;
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
UpgradePayloadFromUberToApache upgrader = new UpgradePayloadFromUberToApache(cfg);
|
||||
upgrader.run();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,146 @@
|
||||
/*
|
||||
* 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.utilities.deltastreamer;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Function;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Base Class for running delta-sync/compaction in separate thread and controlling their life-cyle
|
||||
*/
|
||||
public abstract class AbstractDeltaStreamerService implements Serializable {
|
||||
|
||||
protected static volatile Logger log = LogManager.getLogger(AbstractDeltaStreamerService.class);
|
||||
|
||||
// Flag to track if the service is started.
|
||||
private boolean started;
|
||||
// Flag indicating shutdown is externally requested
|
||||
private boolean shutdownRequested;
|
||||
// Flag indicating the service is shutdown
|
||||
private volatile boolean shutdown;
|
||||
// Executor Service for running delta-sync/compaction
|
||||
private transient ExecutorService executor;
|
||||
// Future tracking delta-sync/compaction
|
||||
private transient CompletableFuture future;
|
||||
|
||||
AbstractDeltaStreamerService() {
|
||||
shutdownRequested = false;
|
||||
}
|
||||
|
||||
boolean isShutdownRequested() {
|
||||
return shutdownRequested;
|
||||
}
|
||||
|
||||
boolean isShutdown() {
|
||||
return shutdown;
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait till the service shutdown. If the service shutdown with exception, it will be thrown
|
||||
* @throws ExecutionException
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
void waitForShutdown() throws ExecutionException, InterruptedException {
|
||||
try {
|
||||
future.get();
|
||||
} catch (ExecutionException ex) {
|
||||
log.error("Service shutdown with error", ex);
|
||||
throw ex;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Request shutdown either forcefully or gracefully. Graceful shutdown allows the service to finish up the current
|
||||
* round of work and shutdown. For graceful shutdown, it waits till the service is shutdown
|
||||
* @param force Forcefully shutdown
|
||||
*/
|
||||
void shutdown(boolean force) {
|
||||
if (!shutdownRequested || force) {
|
||||
shutdownRequested = true;
|
||||
if (executor != null) {
|
||||
if (force) {
|
||||
executor.shutdownNow();
|
||||
} else {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// Wait for some max time after requesting shutdown
|
||||
executor.awaitTermination(24, TimeUnit.HOURS);
|
||||
} catch (InterruptedException ie) {
|
||||
log.error("Interrupted while waiting for shutdown", ie);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Start the service. Runs the service in a different thread and returns. Also starts a monitor thread
|
||||
* to run-callbacks in case of shutdown
|
||||
* @param onShutdownCallback
|
||||
*/
|
||||
public void start(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
Pair<CompletableFuture, ExecutorService> res = startService();
|
||||
future = res.getKey();
|
||||
executor = res.getValue();
|
||||
started = true;
|
||||
monitorThreads(onShutdownCallback);
|
||||
}
|
||||
|
||||
/**
|
||||
* Service implementation
|
||||
* @return
|
||||
*/
|
||||
protected abstract Pair<CompletableFuture, ExecutorService> startService();
|
||||
|
||||
/**
|
||||
* A monitor thread is started which would trigger a callback if the service is shutdown
|
||||
* @param onShutdownCallback
|
||||
*/
|
||||
private void monitorThreads(Function<Boolean, Boolean> onShutdownCallback) {
|
||||
log.info("Submitting monitor thread !!");
|
||||
Executors.newSingleThreadExecutor().submit(() -> {
|
||||
boolean error = false;
|
||||
try {
|
||||
log.info("Monitoring thread(s) !!");
|
||||
future.get();
|
||||
} catch (ExecutionException ex) {
|
||||
log.error("Monitor noticed one or more threads failed."
|
||||
+ " Requesting graceful shutdown of other threads", ex);
|
||||
error = true;
|
||||
shutdown(false);
|
||||
} catch (InterruptedException ie) {
|
||||
log.error("Got interrupted Monitoring threads", ie);
|
||||
error = true;
|
||||
shutdown(false);
|
||||
} finally {
|
||||
// Mark as shutdown
|
||||
shutdown = true;
|
||||
onShutdownCallback.apply(error);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.utilities.deltastreamer;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Run one round of compaction
|
||||
*/
|
||||
public class Compactor implements Serializable {
|
||||
|
||||
protected static volatile Logger log = LogManager.getLogger(Compactor.class);
|
||||
|
||||
private transient HoodieWriteClient compactionClient;
|
||||
private transient JavaSparkContext jssc;
|
||||
|
||||
public Compactor(HoodieWriteClient compactionClient, JavaSparkContext jssc) {
|
||||
this.jssc = jssc;
|
||||
this.compactionClient = compactionClient;
|
||||
}
|
||||
|
||||
public void compact(HoodieInstant instant) throws IOException {
|
||||
log.info("Compactor executing compaction " + instant);
|
||||
JavaRDD<WriteStatus> res = compactionClient.compact(instant.getTimestamp());
|
||||
long numWriteErrors = res.collect().stream().filter(r -> r.hasErrors()).count();
|
||||
if (numWriteErrors != 0) {
|
||||
// We treat even a single error in compaction as fatal
|
||||
log.error("Compaction for instant (" + instant + ") failed with write errors. "
|
||||
+ "Errors :" + numWriteErrors);
|
||||
throw new HoodieException("Compaction for instant (" + instant + ") failed with write errors. "
|
||||
+ "Errors :" + numWriteErrors);
|
||||
}
|
||||
// Commit compaction
|
||||
compactionClient.commitCompaction(instant.getTimestamp(), res, Option.empty());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,505 @@
|
||||
/*
|
||||
* 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.utilities.deltastreamer;
|
||||
|
||||
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
|
||||
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.KeyGenerator;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.Operation;
|
||||
import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException;
|
||||
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.InputBatch;
|
||||
import org.apache.hudi.utilities.transform.Transformer;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import scala.collection.JavaConversions;
|
||||
|
||||
|
||||
/**
|
||||
* Sync's one batch of data to hoodie dataset
|
||||
*/
|
||||
public class DeltaSync implements Serializable {
|
||||
|
||||
protected static volatile Logger log = LogManager.getLogger(DeltaSync.class);
|
||||
public static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
|
||||
public static String CHECKPOINT_RESET_KEY = "deltastreamer.checkpoint.reset_key";
|
||||
|
||||
/**
|
||||
* Delta Sync Config
|
||||
*/
|
||||
private final HoodieDeltaStreamer.Config cfg;
|
||||
|
||||
/**
|
||||
* Source to pull deltas from
|
||||
*/
|
||||
private transient SourceFormatAdapter formatAdapter;
|
||||
|
||||
/**
|
||||
* Schema provider that supplies the command for reading the input and writing out the target table.
|
||||
*/
|
||||
private transient SchemaProvider schemaProvider;
|
||||
|
||||
/**
|
||||
* Allows transforming source to target dataset before writing
|
||||
*/
|
||||
private transient Transformer transformer;
|
||||
|
||||
/**
|
||||
* Extract the key for the target dataset
|
||||
*/
|
||||
private KeyGenerator keyGenerator;
|
||||
|
||||
/**
|
||||
* Filesystem used
|
||||
*/
|
||||
private transient FileSystem fs;
|
||||
|
||||
/**
|
||||
* Spark context
|
||||
*/
|
||||
private transient JavaSparkContext jssc;
|
||||
|
||||
/**
|
||||
* Spark Session
|
||||
*/
|
||||
private transient SparkSession sparkSession;
|
||||
|
||||
/**
|
||||
* Hive Config
|
||||
*/
|
||||
private transient HiveConf hiveConf;
|
||||
|
||||
/**
|
||||
* Bag of properties with source, hoodie client, key generator etc.
|
||||
*/
|
||||
private final TypedProperties props;
|
||||
|
||||
/**
|
||||
* Callback when write client is instantiated
|
||||
*/
|
||||
private transient Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient;
|
||||
|
||||
/**
|
||||
* Timeline with completed commits
|
||||
*/
|
||||
private transient Option<HoodieTimeline> commitTimelineOpt;
|
||||
|
||||
/**
|
||||
* Write Client
|
||||
*/
|
||||
private transient HoodieWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* Table Type
|
||||
*/
|
||||
private final HoodieTableType tableType;
|
||||
|
||||
|
||||
public DeltaSync(HoodieDeltaStreamer.Config cfg, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider, HoodieTableType tableType, TypedProperties props,
|
||||
JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf,
|
||||
Function<HoodieWriteClient, Boolean> onInitializingHoodieWriteClient)
|
||||
throws IOException {
|
||||
|
||||
this.cfg = cfg;
|
||||
this.jssc = jssc;
|
||||
this.sparkSession = sparkSession;
|
||||
this.fs = fs;
|
||||
this.tableType = tableType;
|
||||
this.onInitializingHoodieWriteClient = onInitializingHoodieWriteClient;
|
||||
this.props = props;
|
||||
log.info("Creating delta streamer with configs : " + props.toString());
|
||||
this.schemaProvider = schemaProvider;
|
||||
|
||||
refreshTimeline();
|
||||
|
||||
this.transformer = UtilHelpers.createTransformer(cfg.transformerClassName);
|
||||
this.keyGenerator = DataSourceUtils.createKeyGenerator(props);
|
||||
|
||||
this.formatAdapter = new SourceFormatAdapter(UtilHelpers.createSource(cfg.sourceClassName, props, jssc,
|
||||
sparkSession, schemaProvider));
|
||||
|
||||
this.hiveConf = hiveConf;
|
||||
if (cfg.filterDupes) {
|
||||
cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation;
|
||||
}
|
||||
|
||||
// If schemaRegistry already resolved, setup write-client
|
||||
setupWriteClient();
|
||||
}
|
||||
|
||||
/**
|
||||
* Refresh Timeline
|
||||
*/
|
||||
private void refreshTimeline() throws IOException {
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath);
|
||||
this.commitTimelineOpt = Option.of(meta.getActiveTimeline().getCommitsTimeline()
|
||||
.filterCompletedInstants());
|
||||
} else {
|
||||
this.commitTimelineOpt = Option.empty();
|
||||
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath,
|
||||
cfg.storageType, cfg.targetTableName, "archived");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Run one round of delta sync and return new compaction instant if one got scheduled
|
||||
*/
|
||||
public Option<String> syncOnce() throws Exception {
|
||||
Option<String> scheduledCompaction = Option.empty();
|
||||
HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(schemaProvider));
|
||||
Timer.Context overallTimerContext = metrics.getOverallTimerContext();
|
||||
|
||||
// Refresh Timeline
|
||||
refreshTimeline();
|
||||
|
||||
Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> srcRecordsWithCkpt =
|
||||
readFromSource(commitTimelineOpt);
|
||||
|
||||
if (null != srcRecordsWithCkpt) {
|
||||
// this is the first input batch. If schemaProvider not set, use it and register Avro Schema and start
|
||||
// compactor
|
||||
if (null == schemaProvider) {
|
||||
// Set the schemaProvider if not user-provided
|
||||
this.schemaProvider = srcRecordsWithCkpt.getKey();
|
||||
// Setup HoodieWriteClient and compaction now that we decided on schema
|
||||
setupWriteClient();
|
||||
}
|
||||
|
||||
scheduledCompaction = writeToSink(srcRecordsWithCkpt.getRight().getRight(),
|
||||
srcRecordsWithCkpt.getRight().getLeft(), metrics, overallTimerContext);
|
||||
}
|
||||
|
||||
// Clear persistent RDDs
|
||||
jssc.getPersistentRDDs().values().forEach(JavaRDD::unpersist);
|
||||
return scheduledCompaction;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read from Upstream Source and apply transformation if needed
|
||||
*/
|
||||
private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource(
|
||||
Option<HoodieTimeline> commitTimelineOpt) throws Exception {
|
||||
// Retrieve the previous round checkpoints, if any
|
||||
Option<String> resumeCheckpointStr = Option.empty();
|
||||
if (commitTimelineOpt.isPresent()) {
|
||||
Option<HoodieInstant> lastCommit = commitTimelineOpt.get().lastInstant();
|
||||
if (lastCommit.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get(), HoodieCommitMetadata.class);
|
||||
if (cfg.checkpoint != null && !cfg.checkpoint.equals(commitMetadata.getMetadata(CHECKPOINT_RESET_KEY))) {
|
||||
resumeCheckpointStr = Option.of(cfg.checkpoint);
|
||||
} else if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) {
|
||||
resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY));
|
||||
} else {
|
||||
throw new HoodieDeltaStreamerException(
|
||||
"Unable to find previous checkpoint. Please double check if this table "
|
||||
+ "was indeed built via delta streamer ");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath,
|
||||
cfg.storageType, cfg.targetTableName, "archived");
|
||||
}
|
||||
|
||||
if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) {
|
||||
resumeCheckpointStr = Option.of(cfg.checkpoint);
|
||||
}
|
||||
log.info("Checkpoint to resume from : " + resumeCheckpointStr);
|
||||
|
||||
final Option<JavaRDD<GenericRecord>> avroRDDOptional;
|
||||
final String checkpointStr;
|
||||
final SchemaProvider schemaProvider;
|
||||
if (transformer != null) {
|
||||
// Transformation is needed. Fetch New rows in Row Format, apply transformation and then convert them
|
||||
// to generic records for writing
|
||||
InputBatch<Dataset<Row>> dataAndCheckpoint = formatAdapter.fetchNewDataInRowFormat(
|
||||
resumeCheckpointStr, cfg.sourceLimit);
|
||||
|
||||
Option<Dataset<Row>> transformed =
|
||||
dataAndCheckpoint.getBatch().map(data -> transformer.apply(jssc, sparkSession, data, props));
|
||||
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
|
||||
avroRDDOptional = transformed.map(t ->
|
||||
AvroConversionUtils.createRdd(t, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()
|
||||
);
|
||||
// Use Transformed Row's schema if not overridden
|
||||
schemaProvider =
|
||||
this.schemaProvider == null ? transformed.map(r -> (SchemaProvider) new RowBasedSchemaProvider(r.schema()))
|
||||
.orElse(dataAndCheckpoint.getSchemaProvider()) : this.schemaProvider;
|
||||
} else {
|
||||
// Pull the data from the source & prepare the write
|
||||
InputBatch<JavaRDD<GenericRecord>> dataAndCheckpoint =
|
||||
formatAdapter.fetchNewDataInAvroFormat(resumeCheckpointStr, cfg.sourceLimit);
|
||||
avroRDDOptional = dataAndCheckpoint.getBatch();
|
||||
checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch();
|
||||
schemaProvider = dataAndCheckpoint.getSchemaProvider();
|
||||
}
|
||||
|
||||
if ((!avroRDDOptional.isPresent()) || (avroRDDOptional.get().isEmpty())) {
|
||||
log.info("No new data, nothing to commit.. ");
|
||||
return null;
|
||||
}
|
||||
|
||||
JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get();
|
||||
JavaRDD<HoodieRecord> records = avroRDD.map(gr -> {
|
||||
HoodieRecordPayload payload = DataSourceUtils.createPayload(cfg.payloadClassName, gr,
|
||||
(Comparable) gr.get(cfg.sourceOrderingField));
|
||||
return new HoodieRecord<>(keyGenerator.getKey(gr), payload);
|
||||
});
|
||||
return Pair.of(schemaProvider, Pair.of(checkpointStr, records));
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform Hoodie Write. Run Cleaner, schedule compaction and syncs to hive if needed
|
||||
*
|
||||
* @param records Input Records
|
||||
* @param checkpointStr Checkpoint String
|
||||
* @param metrics Metrics
|
||||
* @return Option Compaction instant if one is scheduled
|
||||
*/
|
||||
private Option<String> writeToSink(JavaRDD<HoodieRecord> records, String checkpointStr,
|
||||
HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception {
|
||||
|
||||
Option<String> scheduledCompactionInstant = Option.empty();
|
||||
|
||||
// filter dupes if needed
|
||||
if (cfg.filterDupes) {
|
||||
// turn upserts to insert
|
||||
cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation;
|
||||
records = DataSourceUtils.dropDuplicates(jssc, records, writeClient.getConfig(),
|
||||
writeClient.getTimelineServer());
|
||||
|
||||
if (records.isEmpty()) {
|
||||
log.info("No new data, nothing to commit.. ");
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
String commitTime = startCommit();
|
||||
log.info("Starting commit : " + commitTime);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD;
|
||||
if (cfg.operation == Operation.INSERT) {
|
||||
writeStatusRDD = writeClient.insert(records, commitTime);
|
||||
} else if (cfg.operation == Operation.UPSERT) {
|
||||
writeStatusRDD = writeClient.upsert(records, commitTime);
|
||||
} else if (cfg.operation == Operation.BULK_INSERT) {
|
||||
writeStatusRDD = writeClient.bulkInsert(records, commitTime);
|
||||
} else {
|
||||
throw new HoodieDeltaStreamerException("Unknown operation :" + cfg.operation);
|
||||
}
|
||||
|
||||
long totalErrorRecords = writeStatusRDD.mapToDouble(ws -> ws.getTotalErrorRecords()).sum().longValue();
|
||||
long totalRecords = writeStatusRDD.mapToDouble(ws -> ws.getTotalRecords()).sum().longValue();
|
||||
boolean hasErrors = totalErrorRecords > 0;
|
||||
long hiveSyncTimeMs = 0;
|
||||
if (!hasErrors || cfg.commitOnErrors) {
|
||||
HashMap<String, String> checkpointCommitMetadata = new HashMap<>();
|
||||
checkpointCommitMetadata.put(CHECKPOINT_KEY, checkpointStr);
|
||||
if (cfg.checkpoint != null) {
|
||||
checkpointCommitMetadata.put(CHECKPOINT_RESET_KEY, cfg.checkpoint);
|
||||
}
|
||||
|
||||
if (hasErrors) {
|
||||
log.warn("Some records failed to be merged but forcing commit since commitOnErrors set. Errors/Total="
|
||||
+ totalErrorRecords + "/" + totalRecords);
|
||||
}
|
||||
|
||||
boolean success = writeClient.commit(commitTime, writeStatusRDD,
|
||||
Option.of(checkpointCommitMetadata));
|
||||
if (success) {
|
||||
log.info("Commit " + commitTime + " successful!");
|
||||
|
||||
// Schedule compaction if needed
|
||||
if (cfg.isAsyncCompactionEnabled()) {
|
||||
scheduledCompactionInstant = writeClient.scheduleCompaction(Option.of(checkpointCommitMetadata));
|
||||
}
|
||||
|
||||
// Sync to hive if enabled
|
||||
Timer.Context hiveSyncContext = metrics.getHiveSyncTimerContext();
|
||||
syncHive();
|
||||
hiveSyncTimeMs = hiveSyncContext != null ? hiveSyncContext.stop() : 0;
|
||||
} else {
|
||||
log.info("Commit " + commitTime + " failed!");
|
||||
throw new HoodieException("Commit " + commitTime + " failed!");
|
||||
}
|
||||
} else {
|
||||
log.error("Delta Sync found errors when writing. Errors/Total="
|
||||
+ totalErrorRecords + "/" + totalRecords);
|
||||
log.error("Printing out the top 100 errors");
|
||||
writeStatusRDD.filter(ws -> ws.hasErrors()).take(100).forEach(ws -> {
|
||||
log.error("Global error :", ws.getGlobalError());
|
||||
if (ws.getErrors().size() > 0) {
|
||||
ws.getErrors().entrySet().forEach(r ->
|
||||
log.trace("Error for key:" + r.getKey() + " is " + r.getValue()));
|
||||
}
|
||||
});
|
||||
// Rolling back instant
|
||||
writeClient.rollback(commitTime);
|
||||
throw new HoodieException("Commit " + commitTime + " failed and rolled-back !");
|
||||
}
|
||||
long overallTimeMs = overallTimerContext != null ? overallTimerContext.stop() : 0;
|
||||
|
||||
// Send DeltaStreamer Metrics
|
||||
metrics.updateDeltaStreamerMetrics(overallTimeMs, hiveSyncTimeMs);
|
||||
|
||||
return scheduledCompactionInstant;
|
||||
}
|
||||
|
||||
private String startCommit() {
|
||||
final int maxRetries = 2;
|
||||
int retryNum = 1;
|
||||
RuntimeException lastException = null;
|
||||
while (retryNum <= maxRetries) {
|
||||
try {
|
||||
return writeClient.startCommit();
|
||||
} catch (IllegalArgumentException ie) {
|
||||
lastException = ie;
|
||||
log.error("Got error trying to start a new commit. Retrying after sleeping for a sec", ie);
|
||||
retryNum++;
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
} catch (InterruptedException e) {
|
||||
//No-Op
|
||||
}
|
||||
}
|
||||
}
|
||||
throw lastException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sync to Hive
|
||||
*/
|
||||
private void syncHive() throws ClassNotFoundException {
|
||||
if (cfg.enableHiveSync) {
|
||||
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath);
|
||||
log.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName
|
||||
+ "). Hive metastore URL :" + hiveSyncConfig.jdbcUrl + ", basePath :" + cfg.targetBasePath);
|
||||
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Note that depending on configs and source-type, schemaProvider could either be eagerly or lazily created.
|
||||
* SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of
|
||||
* this constraint.
|
||||
*/
|
||||
public void setupWriteClient() {
|
||||
log.info("Setting up Hoodie Write Client");
|
||||
if ((null != schemaProvider) && (null == writeClient)) {
|
||||
registerAvroSchemas(schemaProvider);
|
||||
HoodieWriteConfig hoodieCfg = getHoodieClientConfig(schemaProvider);
|
||||
writeClient = new HoodieWriteClient<>(jssc, hoodieCfg, true);
|
||||
onInitializingHoodieWriteClient.apply(writeClient);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to construct Write Client config
|
||||
*
|
||||
* @param schemaProvider Schema Provider
|
||||
*/
|
||||
private HoodieWriteConfig getHoodieClientConfig(SchemaProvider schemaProvider) {
|
||||
HoodieWriteConfig.Builder builder =
|
||||
HoodieWriteConfig.newBuilder()
|
||||
.withProps(props)
|
||||
.withPath(cfg.targetBasePath)
|
||||
.combineInput(cfg.filterDupes, true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withPayloadClass(cfg.payloadClassName)
|
||||
// Inline compaction is disabled for continuous mode. otherwise enabled for MOR
|
||||
.withInlineCompaction(cfg.isInlineCompactionEnabled()).build())
|
||||
.forTable(cfg.targetTableName)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withAutoCommit(false);
|
||||
if (null != schemaProvider) {
|
||||
builder = builder.withSchema(schemaProvider.getTargetSchema().toString());
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Register Avro Schemas
|
||||
*
|
||||
* @param schemaProvider Schema Provider
|
||||
*/
|
||||
private void registerAvroSchemas(SchemaProvider schemaProvider) {
|
||||
// register the schemas, so that shuffle does not serialize the full schemas
|
||||
if (null != schemaProvider) {
|
||||
List<Schema> schemas = Arrays.asList(schemaProvider.getSourceSchema(), schemaProvider.getTargetSchema());
|
||||
log.info("Registering Schema :" + schemas);
|
||||
jssc.sc().getConf().registerAvroSchemas(JavaConversions.asScalaBuffer(schemas).toList());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Close all resources
|
||||
*/
|
||||
public void close() {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,572 @@
|
||||
/*
|
||||
* 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.utilities.deltastreamer;
|
||||
|
||||
import com.beust.jcommander.IStringConverter;
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.beust.jcommander.ParameterException;
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.OverwriteWithLatestAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.utilities.HiveIncrementalPuller;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.JsonDFSSource;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
|
||||
/**
|
||||
* An Utility which can incrementally take the output from {@link HiveIncrementalPuller} and apply it to the target
|
||||
* dataset. Does not maintain any state, queries at runtime to see how far behind the target dataset is from the source
|
||||
* dataset. This can be overriden to force sync from a timestamp.
|
||||
*
|
||||
* In continuous mode, DeltaStreamer runs in loop-mode going through the below operations
|
||||
* (a) pull-from-source
|
||||
* (b) write-to-sink
|
||||
* (c) Schedule Compactions if needed
|
||||
* (d) Conditionally Sync to Hive
|
||||
* each cycle. For MOR table with continuous mode enabled, a seperate compactor thread is allocated to execute
|
||||
* compactions
|
||||
*/
|
||||
public class HoodieDeltaStreamer implements Serializable {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(HoodieDeltaStreamer.class);
|
||||
|
||||
public static String CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
|
||||
|
||||
private final transient Config cfg;
|
||||
|
||||
private transient DeltaSyncService deltaSyncService;
|
||||
|
||||
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc) throws IOException {
|
||||
this(cfg, jssc, FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration()),
|
||||
getDefaultHiveConf(jssc.hadoopConfiguration()));
|
||||
}
|
||||
|
||||
public HoodieDeltaStreamer(Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf) throws IOException {
|
||||
this.cfg = cfg;
|
||||
this.deltaSyncService = new DeltaSyncService(cfg, jssc, fs, hiveConf);
|
||||
}
|
||||
|
||||
public void shutdownGracefully() {
|
||||
deltaSyncService.shutdown(false);
|
||||
}
|
||||
|
||||
private static HiveConf getDefaultHiveConf(Configuration cfg) {
|
||||
HiveConf hiveConf = new HiveConf();
|
||||
hiveConf.addResource(cfg);
|
||||
return hiveConf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Main method to start syncing
|
||||
* @throws Exception
|
||||
*/
|
||||
public void sync() throws Exception {
|
||||
if (cfg.continuousMode) {
|
||||
deltaSyncService.start(this::onDeltaSyncShutdown);
|
||||
deltaSyncService.waitForShutdown();
|
||||
log.info("Delta Sync shutting down");
|
||||
} else {
|
||||
log.info("Delta Streamer running only single round");
|
||||
deltaSyncService.getDeltaSync().syncOnce();
|
||||
deltaSyncService.close();
|
||||
log.info("Shut down deltastreamer");
|
||||
}
|
||||
}
|
||||
|
||||
private boolean onDeltaSyncShutdown(boolean error) {
|
||||
log.info("DeltaSync shutdown. Closing write client. Error?" + error);
|
||||
deltaSyncService.close();
|
||||
return true;
|
||||
}
|
||||
|
||||
public enum Operation {
|
||||
UPSERT, INSERT, BULK_INSERT
|
||||
}
|
||||
|
||||
private static class OperationConvertor implements IStringConverter<Operation> {
|
||||
|
||||
@Override
|
||||
public Operation convert(String value) throws ParameterException {
|
||||
return Operation.valueOf(value);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--target-base-path"}, description = "base path for the target hoodie dataset. "
|
||||
+ "(Will be created if did not exist first time around. If exists, expected to be a hoodie dataset)",
|
||||
required = true)
|
||||
public String targetBasePath;
|
||||
|
||||
// TODO: How to obtain hive configs to register?
|
||||
@Parameter(names = {"--target-table"}, description = "name of the target table in Hive", required = true)
|
||||
public String targetTableName;
|
||||
|
||||
@Parameter(names = {"--storage-type"}, description = "Type of Storage. "
|
||||
+ "COPY_ON_WRITE (or) MERGE_ON_READ", required = true)
|
||||
public String storageType;
|
||||
|
||||
@Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for "
|
||||
+ "hoodie client, schema provider, key generator and data source. For hoodie client props, sane defaults are "
|
||||
+ "used, but recommend use to provide basic things like metrics endpoints, hive configs etc. For sources, refer"
|
||||
+ "to individual classes, for supported properties.")
|
||||
public String propsFilePath =
|
||||
"file://" + System.getProperty("user.dir") + "/src/test/resources/delta-streamer-config/dfs-source.properties";
|
||||
|
||||
@Parameter(names = {"--hudi-conf"}, description = "Any configuration that can be set in the properties file "
|
||||
+ "(using the CLI parameter \"--propsFilePath\") can also be passed command line using this parameter")
|
||||
public List<String> configs = new ArrayList<>();
|
||||
|
||||
@Parameter(names = {"--source-class"}, description = "Subclass of org.apache.hudi.utilities.sources to read data. "
|
||||
+ "Built-in options: org.apache.hudi.utilities.sources.{JsonDFSSource (default), AvroDFSSource, "
|
||||
+ "JsonKafkaSource, AvroKafkaSource, HiveIncrPullSource}")
|
||||
public String sourceClassName = JsonDFSSource.class.getName();
|
||||
|
||||
@Parameter(names = {"--source-ordering-field"}, description = "Field within source record to decide how"
|
||||
+ " to break ties between records with same key in input data. Default: 'ts' holding unix timestamp of record")
|
||||
public String sourceOrderingField = "ts";
|
||||
|
||||
@Parameter(names = {"--payload-class"}, description = "subclass of HoodieRecordPayload, that works off "
|
||||
+ "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value")
|
||||
public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
|
||||
|
||||
@Parameter(names = {"--schemaprovider-class"}, description = "subclass of org.apache.hudi.utilities.schema"
|
||||
+ ".SchemaProvider to attach schemas to input & target table data, built in options: "
|
||||
+ "org.apache.hudi.utilities.schema.FilebasedSchemaProvider."
|
||||
+ "Source (See org.apache.hudi.utilities.sources.Source) implementation can implement their own SchemaProvider."
|
||||
+ " For Sources that return Dataset<Row>, the schema is obtained implicitly. "
|
||||
+ "However, this CLI option allows overriding the schemaprovider returned by Source.")
|
||||
public String schemaProviderClassName = null;
|
||||
|
||||
@Parameter(names = {"--transformer-class"},
|
||||
description = "subclass of org.apache.hudi.utilities.transform.Transformer"
|
||||
+ ". Allows transforming raw source dataset to a target dataset (conforming to target schema) before "
|
||||
+ "writing. Default : Not set. E:g - org.apache.hudi.utilities.transform.SqlQueryBasedTransformer (which "
|
||||
+ "allows a SQL query templated to be passed as a transformation function)")
|
||||
public String transformerClassName = null;
|
||||
|
||||
@Parameter(names = {"--source-limit"}, description = "Maximum amount of data to read from source. "
|
||||
+ "Default: No limit For e.g: DFS-Source => max bytes to read, Kafka-Source => max events to read")
|
||||
public long sourceLimit = Long.MAX_VALUE;
|
||||
|
||||
@Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input "
|
||||
+ "is purely new data/inserts to gain speed)",
|
||||
converter = OperationConvertor.class)
|
||||
public Operation operation = Operation.UPSERT;
|
||||
|
||||
@Parameter(names = {"--filter-dupes"}, description = "Should duplicate records from source be dropped/filtered out"
|
||||
+ "before insert/bulk-insert")
|
||||
public Boolean filterDupes = false;
|
||||
|
||||
@Parameter(names = {"--enable-hive-sync"}, description = "Enable syncing to hive")
|
||||
public Boolean enableHiveSync = false;
|
||||
|
||||
@Parameter(names = {"--max-pending-compactions"},
|
||||
description = "Maximum number of outstanding inflight/requested compactions. Delta Sync will not happen unless"
|
||||
+ "outstanding compactions is less than this number")
|
||||
public Integer maxPendingCompactions = 5;
|
||||
|
||||
@Parameter(names = {"--continuous"}, description = "Delta Streamer runs in continuous mode running"
|
||||
+ " source-fetch -> Transform -> Hudi Write in loop")
|
||||
public Boolean continuousMode = false;
|
||||
|
||||
@Parameter(names = {"--spark-master"}, description = "spark master to use.")
|
||||
public String sparkMaster = "local[2]";
|
||||
|
||||
@Parameter(names = {"--commit-on-errors"}, description = "Commit even when some records failed to be written")
|
||||
public Boolean commitOnErrors = false;
|
||||
|
||||
@Parameter(names = {"--delta-sync-scheduling-weight"}, description =
|
||||
"Scheduling weight for delta sync as defined in "
|
||||
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
|
||||
public Integer deltaSyncSchedulingWeight = 1;
|
||||
|
||||
@Parameter(names = {"--compact-scheduling-weight"}, description = "Scheduling weight for compaction as defined in "
|
||||
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
|
||||
public Integer compactSchedulingWeight = 1;
|
||||
|
||||
@Parameter(names = {"--delta-sync-scheduling-minshare"}, description = "Minshare for delta sync as defined in "
|
||||
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
|
||||
public Integer deltaSyncSchedulingMinShare = 0;
|
||||
|
||||
@Parameter(names = {"--compact-scheduling-minshare"}, description = "Minshare for compaction as defined in "
|
||||
+ "https://spark.apache.org/docs/latest/job-scheduling.html")
|
||||
public Integer compactSchedulingMinShare = 0;
|
||||
|
||||
/**
|
||||
* Compaction is enabled for MoR table by default. This flag disables it
|
||||
*/
|
||||
@Parameter(names = {"--disable-compaction"}, description = "Compaction is enabled for MoR table by default."
|
||||
+ "This flag disables it ")
|
||||
public Boolean forceDisableCompaction = false;
|
||||
|
||||
/**
|
||||
* Resume Delta Streamer from this checkpoint.
|
||||
*/
|
||||
@Parameter(names = {"--checkpoint"}, description = "Resume Delta Streamer from this checkpoint.")
|
||||
public String checkpoint = null;
|
||||
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
|
||||
public boolean isAsyncCompactionEnabled() {
|
||||
return continuousMode && !forceDisableCompaction
|
||||
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType));
|
||||
}
|
||||
|
||||
public boolean isInlineCompactionEnabled() {
|
||||
return !continuousMode && !forceDisableCompaction
|
||||
&& HoodieTableType.MERGE_ON_READ.equals(HoodieTableType.valueOf(storageType));
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
|
||||
Map<String, String> additionalSparkConfigs = SchedulerConfGenerator.getSparkSchedulingConfigs(cfg);
|
||||
JavaSparkContext jssc = UtilHelpers.buildSparkContext("delta-streamer-" + cfg.targetTableName,
|
||||
cfg.sparkMaster, additionalSparkConfigs);
|
||||
try {
|
||||
new HoodieDeltaStreamer(cfg, jssc).sync();
|
||||
} finally {
|
||||
jssc.stop();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Syncs data either in single-run or in continuous mode.
|
||||
*/
|
||||
public static class DeltaSyncService extends AbstractDeltaStreamerService {
|
||||
|
||||
/**
|
||||
* Delta Sync Config
|
||||
*/
|
||||
private final HoodieDeltaStreamer.Config cfg;
|
||||
|
||||
/**
|
||||
* Schema provider that supplies the command for reading the input and writing out the target table.
|
||||
*/
|
||||
private transient SchemaProvider schemaProvider;
|
||||
|
||||
/**
|
||||
* Spark Session
|
||||
*/
|
||||
private transient SparkSession sparkSession;
|
||||
|
||||
/**
|
||||
* Spark context
|
||||
*/
|
||||
private transient JavaSparkContext jssc;
|
||||
|
||||
/**
|
||||
* Bag of properties with source, hoodie client, key generator etc.
|
||||
*/
|
||||
TypedProperties props;
|
||||
|
||||
/**
|
||||
* Async Compactor Service
|
||||
*/
|
||||
private AsyncCompactService asyncCompactService;
|
||||
|
||||
/**
|
||||
* Table Type
|
||||
*/
|
||||
private final HoodieTableType tableType;
|
||||
|
||||
/**
|
||||
* Delta Sync
|
||||
*/
|
||||
private transient DeltaSync deltaSync;
|
||||
|
||||
public DeltaSyncService(HoodieDeltaStreamer.Config cfg, JavaSparkContext jssc, FileSystem fs, HiveConf hiveConf)
|
||||
throws IOException {
|
||||
this.cfg = cfg;
|
||||
this.jssc = jssc;
|
||||
this.sparkSession = SparkSession.builder().config(jssc.getConf()).getOrCreate();
|
||||
|
||||
if (fs.exists(new Path(cfg.targetBasePath))) {
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(
|
||||
new Configuration(fs.getConf()), cfg.targetBasePath, false);
|
||||
tableType = meta.getTableType();
|
||||
// This will guarantee there is no surprise with table type
|
||||
Preconditions.checkArgument(tableType.equals(HoodieTableType.valueOf(cfg.storageType)),
|
||||
"Hoodie table is of type " + tableType + " but passed in CLI argument is " + cfg.storageType);
|
||||
} else {
|
||||
tableType = HoodieTableType.valueOf(cfg.storageType);
|
||||
}
|
||||
|
||||
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
|
||||
log.info("Creating delta streamer with configs : " + props.toString());
|
||||
this.schemaProvider = UtilHelpers.createSchemaProvider(cfg.schemaProviderClassName, props, jssc);
|
||||
|
||||
if (cfg.filterDupes) {
|
||||
cfg.operation = cfg.operation == Operation.UPSERT ? Operation.INSERT : cfg.operation;
|
||||
}
|
||||
|
||||
deltaSync = new DeltaSync(cfg, sparkSession, schemaProvider, tableType,
|
||||
props, jssc, fs, hiveConf, this::onInitializingWriteClient);
|
||||
}
|
||||
|
||||
public DeltaSync getDeltaSync() {
|
||||
return deltaSync;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Pair<CompletableFuture, ExecutorService> startService() {
|
||||
ExecutorService executor = Executors.newFixedThreadPool(1);
|
||||
return Pair.of(CompletableFuture.supplyAsync(() -> {
|
||||
boolean error = false;
|
||||
if (cfg.isAsyncCompactionEnabled()) {
|
||||
// set Scheduler Pool.
|
||||
log.info("Setting Spark Pool name for delta-sync to " + SchedulerConfGenerator.DELTASYNC_POOL_NAME);
|
||||
jssc.setLocalProperty("spark.scheduler.pool", SchedulerConfGenerator.DELTASYNC_POOL_NAME);
|
||||
}
|
||||
try {
|
||||
while (!isShutdownRequested()) {
|
||||
try {
|
||||
Option<String> scheduledCompactionInstant = deltaSync.syncOnce();
|
||||
if (scheduledCompactionInstant.isPresent()) {
|
||||
log.info("Enqueuing new pending compaction instant (" + scheduledCompactionInstant + ")");
|
||||
asyncCompactService.enqueuePendingCompaction(new HoodieInstant(
|
||||
State.REQUESTED, HoodieTimeline.COMPACTION_ACTION,
|
||||
scheduledCompactionInstant.get()));
|
||||
asyncCompactService.waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("Shutting down delta-sync due to exception", e);
|
||||
error = true;
|
||||
throw new HoodieException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
shutdownCompactor(error);
|
||||
}
|
||||
return true;
|
||||
}, executor), executor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Shutdown compactor as DeltaSync is shutdown
|
||||
*/
|
||||
private void shutdownCompactor(boolean error) {
|
||||
log.info("Delta Sync shutdown. Error ?" + error);
|
||||
if (asyncCompactService != null) {
|
||||
log.warn("Gracefully shutting down compactor");
|
||||
asyncCompactService.shutdown(false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback to initialize write client and start compaction service if required
|
||||
* @param writeClient HoodieWriteClient
|
||||
* @return
|
||||
*/
|
||||
protected Boolean onInitializingWriteClient(HoodieWriteClient writeClient) {
|
||||
if (cfg.isAsyncCompactionEnabled()) {
|
||||
asyncCompactService = new AsyncCompactService(jssc, writeClient);
|
||||
// Enqueue existing pending compactions first
|
||||
HoodieTableMetaClient meta = new HoodieTableMetaClient(
|
||||
new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, true);
|
||||
List<HoodieInstant> pending = CompactionUtils.getPendingCompactionInstantTimes(meta);
|
||||
pending.stream().forEach(hoodieInstant -> asyncCompactService.enqueuePendingCompaction(hoodieInstant));
|
||||
asyncCompactService.start((error) -> {
|
||||
// Shutdown DeltaSync
|
||||
shutdown(false);
|
||||
return true;
|
||||
});
|
||||
try {
|
||||
asyncCompactService.waitTillPendingCompactionsReducesTo(cfg.maxPendingCompactions);
|
||||
} catch (InterruptedException ie) {
|
||||
throw new HoodieException(ie);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Close all resources
|
||||
*/
|
||||
public void close() {
|
||||
if (null != deltaSync) {
|
||||
deltaSync.close();
|
||||
}
|
||||
}
|
||||
|
||||
public SchemaProvider getSchemaProvider() {
|
||||
return schemaProvider;
|
||||
}
|
||||
|
||||
public SparkSession getSparkSession() {
|
||||
return sparkSession;
|
||||
}
|
||||
|
||||
public JavaSparkContext getJavaSparkContext() {
|
||||
return jssc;
|
||||
}
|
||||
|
||||
public AsyncCompactService getAsyncCompactService() {
|
||||
return asyncCompactService;
|
||||
}
|
||||
|
||||
public TypedProperties getProps() {
|
||||
return props;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Async Compactor Service tha runs in separate thread. Currently, only one compactor is allowed to run at any time.
|
||||
*/
|
||||
public static class AsyncCompactService extends AbstractDeltaStreamerService {
|
||||
|
||||
private final int maxConcurrentCompaction;
|
||||
private transient Compactor compactor;
|
||||
private transient JavaSparkContext jssc;
|
||||
private transient BlockingQueue<HoodieInstant> pendingCompactions = new LinkedBlockingQueue<>();
|
||||
private transient ReentrantLock queueLock = new ReentrantLock();
|
||||
private transient Condition consumed = queueLock.newCondition();
|
||||
|
||||
public AsyncCompactService(JavaSparkContext jssc, HoodieWriteClient client) {
|
||||
this.jssc = jssc;
|
||||
this.compactor = new Compactor(client, jssc);
|
||||
//TODO: HUDI-157 : Only allow 1 compactor to run in parallel till Incremental View on MOR is fully implemented.
|
||||
this.maxConcurrentCompaction = 1;
|
||||
}
|
||||
|
||||
/**
|
||||
* Enqueues new Pending compaction
|
||||
*/
|
||||
public void enqueuePendingCompaction(HoodieInstant instant) {
|
||||
pendingCompactions.add(instant);
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait till outstanding pending compactions reduces to the passed in value
|
||||
* @param numPendingCompactions Maximum pending compactions allowed
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
public void waitTillPendingCompactionsReducesTo(int numPendingCompactions) throws InterruptedException {
|
||||
try {
|
||||
queueLock.lock();
|
||||
while (!isShutdown() && (pendingCompactions.size() > numPendingCompactions)) {
|
||||
consumed.await();
|
||||
}
|
||||
} finally {
|
||||
queueLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch Next pending compaction if available
|
||||
* @return
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
private HoodieInstant fetchNextCompactionInstant() throws InterruptedException {
|
||||
log.info("Compactor waiting for next instant for compaction upto 60 seconds");
|
||||
HoodieInstant instant = pendingCompactions.poll(60, TimeUnit.SECONDS);
|
||||
if (instant != null) {
|
||||
try {
|
||||
queueLock.lock();
|
||||
// Signal waiting thread
|
||||
consumed.signal();
|
||||
} finally {
|
||||
queueLock.unlock();
|
||||
}
|
||||
}
|
||||
return instant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start Compaction Service
|
||||
*/
|
||||
protected Pair<CompletableFuture, ExecutorService> startService() {
|
||||
ExecutorService executor = Executors.newFixedThreadPool(maxConcurrentCompaction);
|
||||
List<CompletableFuture<Boolean>> compactionFutures =
|
||||
IntStream.range(0, maxConcurrentCompaction).mapToObj(i -> CompletableFuture.supplyAsync(() -> {
|
||||
try {
|
||||
// Set Compactor Pool Name for allowing users to prioritize compaction
|
||||
log.info("Setting Spark Pool name for compaction to " + SchedulerConfGenerator.COMPACT_POOL_NAME);
|
||||
jssc.setLocalProperty("spark.scheduler.pool", SchedulerConfGenerator.COMPACT_POOL_NAME);
|
||||
|
||||
while (!isShutdownRequested()) {
|
||||
final HoodieInstant instant = fetchNextCompactionInstant();
|
||||
if (null != instant) {
|
||||
compactor.compact(instant);
|
||||
}
|
||||
}
|
||||
log.info("Compactor shutting down properly!!");
|
||||
} catch (InterruptedException ie) {
|
||||
log.warn("Compactor executor thread got interrupted exception. Stopping", ie);
|
||||
} catch (IOException e) {
|
||||
log.error("Compactor executor failed", e);
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return true;
|
||||
}, executor)).collect(Collectors.toList());
|
||||
return Pair.of(CompletableFuture.allOf(compactionFutures.stream().toArray(CompletableFuture[]::new)), executor);
|
||||
}
|
||||
}
|
||||
|
||||
public DeltaSyncService getDeltaSyncService() {
|
||||
return deltaSyncService;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* 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.utilities.deltastreamer;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.metrics.Metrics;
|
||||
|
||||
public class HoodieDeltaStreamerMetrics {
|
||||
|
||||
private HoodieWriteConfig config = null;
|
||||
private String tableName = null;
|
||||
|
||||
public String overallTimerName = null;
|
||||
public String hiveSyncTimerName = null;
|
||||
private Timer overallTimer = null;
|
||||
public Timer hiveSyncTimer = null;
|
||||
|
||||
public HoodieDeltaStreamerMetrics(HoodieWriteConfig config) {
|
||||
this.config = config;
|
||||
this.tableName = config.getTableName();
|
||||
if (config.isMetricsOn()) {
|
||||
Metrics.init(config);
|
||||
this.overallTimerName = getMetricsName("timer", "deltastreamer");
|
||||
this.hiveSyncTimerName = getMetricsName("timer", "deltastreamerHiveSync");
|
||||
}
|
||||
}
|
||||
|
||||
public Timer.Context getOverallTimerContext() {
|
||||
if (config.isMetricsOn() && overallTimer == null) {
|
||||
overallTimer = createTimer(overallTimerName);
|
||||
}
|
||||
return overallTimer == null ? null : overallTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getHiveSyncTimerContext() {
|
||||
if (config.isMetricsOn() && hiveSyncTimer == null) {
|
||||
hiveSyncTimer = createTimer(hiveSyncTimerName);
|
||||
}
|
||||
return hiveSyncTimer == null ? null : hiveSyncTimer.time();
|
||||
}
|
||||
|
||||
private Timer createTimer(String name) {
|
||||
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
||||
}
|
||||
|
||||
String getMetricsName(String action, String metric) {
|
||||
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
|
||||
}
|
||||
|
||||
public void updateDeltaStreamerMetrics(long durationInNs, long hiveSyncNs) {
|
||||
if (config.isMetricsOn()) {
|
||||
Metrics.registerGauge(getMetricsName("deltastreamer", "duration"), getDurationInMs(durationInNs));
|
||||
Metrics.registerGauge(getMetricsName("deltastreamer", "hiveSyncDuration"), getDurationInMs(hiveSyncNs));
|
||||
}
|
||||
}
|
||||
|
||||
public long getDurationInMs(long ctxDuration) {
|
||||
return ctxDuration / 1000000;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,127 @@
|
||||
/*
|
||||
* Copyright (c) 2019 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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.utilities.deltastreamer;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import org.apache.commons.lang.text.StrSubstitutor;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
|
||||
/**
|
||||
* Utility Class to generate Spark Scheduling allocation file. This kicks in only when user
|
||||
* sets spark.scheduler.mode=FAIR at spark-submit time
|
||||
*/
|
||||
public class SchedulerConfGenerator {
|
||||
|
||||
protected static volatile Logger log = LogManager.getLogger(SchedulerConfGenerator.class);
|
||||
|
||||
public static final String DELTASYNC_POOL_NAME = "hoodiedeltasync";
|
||||
public static final String COMPACT_POOL_NAME = "hoodiecompact";
|
||||
public static final String SPARK_SCHEDULER_MODE_KEY = "spark.scheduler.mode";
|
||||
public static final String SPARK_SCHEDULER_ALLOCATION_FILE_KEY = "spark.scheduler.allocation.file";
|
||||
|
||||
|
||||
private static final String DELTASYNC_POOL_KEY = "deltasync_pool";
|
||||
private static final String COMPACT_POOL_KEY = "compact_pool";
|
||||
private static final String DELTASYNC_POLICY_KEY = "deltasync_policy";
|
||||
private static final String COMPACT_POLICY_KEY = "compact_policy";
|
||||
private static final String DELTASYNC_WEIGHT_KEY = "deltasync_weight";
|
||||
private static final String DELTASYNC_MINSHARE_KEY = "deltasync_minshare";
|
||||
private static final String COMPACT_WEIGHT_KEY = "compact_weight";
|
||||
private static final String COMPACT_MINSHARE_KEY = "compact_minshare";
|
||||
|
||||
private static String SPARK_SCHEDULING_PATTERN =
|
||||
"<?xml version=\"1.0\"?>\n"
|
||||
+ "<allocations>\n"
|
||||
+ " <pool name=\"%(deltasync_pool)\">\n"
|
||||
+ " <schedulingMode>%(deltasync_policy)</schedulingMode>\n"
|
||||
+ " <weight>%(deltasync_weight)</weight>\n"
|
||||
+ " <minShare>%(deltasync_minshare)</minShare>\n"
|
||||
+ " </pool>\n"
|
||||
+ " <pool name=\"%(compact_pool)\">\n"
|
||||
+ " <schedulingMode>%(compact_policy)</schedulingMode>\n"
|
||||
+ " <weight>%(compact_weight)</weight>\n"
|
||||
+ " <minShare>%(compact_minshare)</minShare>\n"
|
||||
+ " </pool>\n"
|
||||
+ "</allocations>";
|
||||
|
||||
private static String generateConfig(Integer deltaSyncWeight, Integer compactionWeight, Integer deltaSyncMinShare,
|
||||
Integer compactionMinShare) {
|
||||
Map<String, String> schedulingProps = new HashMap<>();
|
||||
schedulingProps.put(DELTASYNC_POOL_KEY, DELTASYNC_POOL_NAME);
|
||||
schedulingProps.put(COMPACT_POOL_KEY, COMPACT_POOL_NAME);
|
||||
schedulingProps.put(DELTASYNC_POLICY_KEY, "FAIR");
|
||||
schedulingProps.put(COMPACT_POLICY_KEY, "FAIR");
|
||||
schedulingProps.put(DELTASYNC_WEIGHT_KEY, deltaSyncWeight.toString());
|
||||
schedulingProps.put(DELTASYNC_MINSHARE_KEY, deltaSyncMinShare.toString());
|
||||
schedulingProps.put(COMPACT_WEIGHT_KEY, compactionWeight.toString());
|
||||
schedulingProps.put(COMPACT_MINSHARE_KEY, compactionMinShare.toString());
|
||||
|
||||
StrSubstitutor sub = new StrSubstitutor(schedulingProps, "%(", ")");
|
||||
String xmlString = sub.replace(SPARK_SCHEDULING_PATTERN);
|
||||
log.info("Scheduling Configurations generated. Config=\n" + xmlString);
|
||||
return xmlString;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Helper to set Spark Scheduling Configs dynamically
|
||||
*
|
||||
* @param cfg Config
|
||||
*/
|
||||
public static Map<String, String> getSparkSchedulingConfigs(HoodieDeltaStreamer.Config cfg) throws Exception {
|
||||
scala.Option<String> scheduleModeKeyOption = new SparkConf().getOption(SPARK_SCHEDULER_MODE_KEY);
|
||||
final Option<String> sparkSchedulerMode =
|
||||
scheduleModeKeyOption.isDefined()
|
||||
? Option.of(scheduleModeKeyOption.get())
|
||||
: Option.empty();
|
||||
|
||||
Map<String, String> additionalSparkConfigs = new HashMap<>();
|
||||
if (sparkSchedulerMode.isPresent() && "FAIR".equals(sparkSchedulerMode.get())
|
||||
&& cfg.continuousMode && cfg.storageType.equals(HoodieTableType.MERGE_ON_READ.name())) {
|
||||
String sparkSchedulingConfFile = generateAndStoreConfig(cfg.deltaSyncSchedulingWeight,
|
||||
cfg.compactSchedulingWeight, cfg.deltaSyncSchedulingMinShare, cfg.compactSchedulingMinShare);
|
||||
additionalSparkConfigs.put(SPARK_SCHEDULER_ALLOCATION_FILE_KEY, sparkSchedulingConfFile);
|
||||
} else {
|
||||
log.warn("Job Scheduling Configs will not be in effect as spark.scheduler.mode "
|
||||
+ "is not set to FAIR at instatiation time. Continuing without scheduling configs");
|
||||
}
|
||||
return additionalSparkConfigs;
|
||||
}
|
||||
|
||||
private static String generateAndStoreConfig(Integer deltaSyncWeight,
|
||||
Integer compactionWeight,
|
||||
Integer deltaSyncMinShare,
|
||||
Integer compactionMinShare) throws IOException {
|
||||
File tempConfigFile = File.createTempFile(UUID.randomUUID().toString(), ".xml");
|
||||
BufferedWriter bw = new BufferedWriter(new FileWriter(tempConfigFile));
|
||||
bw.write(generateConfig(deltaSyncWeight, compactionWeight, deltaSyncMinShare, compactionMinShare));
|
||||
bw.close();
|
||||
log.info("Configs written to file" + tempConfigFile.getAbsolutePath());
|
||||
return tempConfigFile.getAbsolutePath();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,112 @@
|
||||
/*
|
||||
* 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.utilities.deltastreamer;
|
||||
|
||||
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE;
|
||||
import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.utilities.sources.AvroSource;
|
||||
import org.apache.hudi.utilities.sources.InputBatch;
|
||||
import org.apache.hudi.utilities.sources.JsonSource;
|
||||
import org.apache.hudi.utilities.sources.RowSource;
|
||||
import org.apache.hudi.utilities.sources.Source;
|
||||
import org.apache.hudi.utilities.sources.helpers.AvroConvertor;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
/**
|
||||
* Adapts data-format provided by the source to the data-format required by the client (DeltaStreamer)
|
||||
*/
|
||||
public final class SourceFormatAdapter {
|
||||
|
||||
private final Source source;
|
||||
|
||||
|
||||
public SourceFormatAdapter(Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch new data in avro format. If the source provides data in different format, they are translated
|
||||
* to Avro format
|
||||
* @param lastCkptStr
|
||||
* @param sourceLimit
|
||||
* @return
|
||||
*/
|
||||
public InputBatch<JavaRDD<GenericRecord>> fetchNewDataInAvroFormat(Option<String> lastCkptStr,
|
||||
long sourceLimit) {
|
||||
switch (source.getSourceType()) {
|
||||
case AVRO:
|
||||
return ((AvroSource)source).fetchNext(lastCkptStr, sourceLimit);
|
||||
case JSON: {
|
||||
InputBatch<JavaRDD<String>> r = ((JsonSource)source).fetchNext(lastCkptStr, sourceLimit);
|
||||
AvroConvertor convertor = new AvroConvertor(r.getSchemaProvider().getSourceSchema());
|
||||
return new InputBatch<>(Option.ofNullable(
|
||||
r.getBatch().map(rdd -> rdd.map(convertor::fromJson))
|
||||
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
|
||||
}
|
||||
case ROW: {
|
||||
InputBatch<Dataset<Row>> r = ((RowSource)source).fetchNext(lastCkptStr, sourceLimit);
|
||||
return new InputBatch<>(Option.ofNullable(r.getBatch().map(
|
||||
rdd -> (AvroConversionUtils.createRdd(rdd, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD()))
|
||||
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
|
||||
}
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch new data in row format. If the source provides data in different format, they are translated
|
||||
* to Row format
|
||||
* @param lastCkptStr
|
||||
* @param sourceLimit
|
||||
* @return
|
||||
*/
|
||||
public InputBatch<Dataset<Row>> fetchNewDataInRowFormat(Option<String> lastCkptStr, long sourceLimit) {
|
||||
switch (source.getSourceType()) {
|
||||
case ROW:
|
||||
return ((RowSource)source).fetchNext(lastCkptStr, sourceLimit);
|
||||
case AVRO: {
|
||||
InputBatch<JavaRDD<GenericRecord>> r = ((AvroSource)source).fetchNext(lastCkptStr, sourceLimit);
|
||||
Schema sourceSchema = r.getSchemaProvider().getSourceSchema();
|
||||
return new InputBatch<>(Option.ofNullable(
|
||||
r.getBatch().map(rdd -> AvroConversionUtils.createDataFrame(JavaRDD.toRDD(rdd),
|
||||
sourceSchema.toString(), source.getSparkSession()))
|
||||
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
|
||||
}
|
||||
case JSON: {
|
||||
InputBatch<JavaRDD<String>> r = ((JsonSource)source).fetchNext(lastCkptStr, sourceLimit);
|
||||
Schema sourceSchema = r.getSchemaProvider().getSourceSchema();
|
||||
StructType dataType = AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema);
|
||||
return new InputBatch<>(Option.ofNullable(
|
||||
r.getBatch().map(rdd -> source.getSparkSession().read().schema(dataType).json(rdd))
|
||||
.orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider());
|
||||
}
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown source type (" + source.getSourceType() + ")");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.utilities.exception;
|
||||
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
public class HoodieDeltaStreamerException extends HoodieException {
|
||||
|
||||
public HoodieDeltaStreamerException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieDeltaStreamerException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* 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.utilities.exception;
|
||||
|
||||
import java.sql.SQLException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
public class HoodieIncrementalPullException extends HoodieException {
|
||||
|
||||
public HoodieIncrementalPullException(String msg, SQLException e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieIncrementalPullException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* 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.utilities.exception;
|
||||
|
||||
import java.sql.SQLException;
|
||||
|
||||
public class HoodieIncrementalPullSQLException extends HoodieIncrementalPullException {
|
||||
|
||||
public HoodieIncrementalPullSQLException(String msg, SQLException e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieIncrementalPullSQLException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,112 @@
|
||||
/*
|
||||
* 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.utilities.keygen;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.TimeZone;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.SimpleKeyGenerator;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException;
|
||||
|
||||
/**
|
||||
* Key generator, that relies on timestamps for partitioning field. Still picks record key by name.
|
||||
*/
|
||||
public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
|
||||
|
||||
enum TimestampType implements Serializable {
|
||||
UNIX_TIMESTAMP, DATE_STRING, MIXED
|
||||
}
|
||||
|
||||
private final TimestampType timestampType;
|
||||
|
||||
private SimpleDateFormat inputDateFormat;
|
||||
|
||||
private final String outputDateFormat;
|
||||
|
||||
|
||||
/**
|
||||
* Supported configs
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
// One value from TimestampType above
|
||||
private static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen"
|
||||
+ ".timebased.timestamp.type";
|
||||
private static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen"
|
||||
+ ".timebased.input"
|
||||
+ ".dateformat";
|
||||
private static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP = "hoodie.deltastreamer.keygen"
|
||||
+ ".timebased.output"
|
||||
+ ".dateformat";
|
||||
}
|
||||
|
||||
public TimestampBasedKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
DataSourceUtils.checkRequiredProperties(config,
|
||||
Arrays.asList(Config.TIMESTAMP_TYPE_FIELD_PROP, Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP));
|
||||
this.timestampType = TimestampType.valueOf(config.getString(Config.TIMESTAMP_TYPE_FIELD_PROP));
|
||||
this.outputDateFormat = config.getString(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP);
|
||||
|
||||
if (timestampType == TimestampType.DATE_STRING || timestampType == TimestampType.MIXED) {
|
||||
DataSourceUtils
|
||||
.checkRequiredProperties(config, Collections.singletonList(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat = new SimpleDateFormat(
|
||||
config.getString(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP));
|
||||
this.inputDateFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieKey getKey(GenericRecord record) {
|
||||
Object partitionVal = DataSourceUtils.getNestedFieldVal(record, partitionPathField);
|
||||
SimpleDateFormat partitionPathFormat = new SimpleDateFormat(outputDateFormat);
|
||||
partitionPathFormat.setTimeZone(TimeZone.getTimeZone("GMT"));
|
||||
|
||||
try {
|
||||
long unixTime;
|
||||
if (partitionVal instanceof Double) {
|
||||
unixTime = ((Double) partitionVal).longValue();
|
||||
} else if (partitionVal instanceof Float) {
|
||||
unixTime = ((Float) partitionVal).longValue();
|
||||
} else if (partitionVal instanceof Long) {
|
||||
unixTime = (Long) partitionVal;
|
||||
} else if (partitionVal instanceof String) {
|
||||
unixTime = inputDateFormat.parse(partitionVal.toString()).getTime() / 1000;
|
||||
} else {
|
||||
throw new HoodieNotSupportedException(
|
||||
"Unexpected type for partition field: " + partitionVal.getClass().getName());
|
||||
}
|
||||
|
||||
return new HoodieKey(DataSourceUtils.getNestedFieldValAsString(record, recordKeyField),
|
||||
partitionPathFormat.format(new Date(unixTime * 1000)));
|
||||
} catch (ParseException pe) {
|
||||
throw new HoodieDeltaStreamerException(
|
||||
"Unable to parse input partition field :" + partitionVal, pe);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,323 @@
|
||||
/*
|
||||
* 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.utilities.perf;
|
||||
|
||||
import com.beust.jcommander.JCommander;
|
||||
import com.beust.jcommander.Parameter;
|
||||
import com.codahale.metrics.Histogram;
|
||||
import com.codahale.metrics.Snapshot;
|
||||
import com.codahale.metrics.UniformReservoir;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ScheduledFuture;
|
||||
import java.util.concurrent.ScheduledThreadPoolExecutor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.table.view.RemoteHoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.timeline.service.TimelineService;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class TimelineServerPerf implements Serializable {
|
||||
|
||||
private static volatile Logger logger = LogManager.getLogger(TimelineServerPerf.class);
|
||||
private final Config cfg;
|
||||
private transient TimelineService timelineServer;
|
||||
private final boolean useExternalTimelineServer;
|
||||
private String hostAddr;
|
||||
|
||||
public TimelineServerPerf(Config cfg) throws IOException {
|
||||
this.cfg = cfg;
|
||||
useExternalTimelineServer = (cfg.serverHost != null);
|
||||
this.timelineServer = new TimelineService(cfg.getTimelinServerConfig());
|
||||
}
|
||||
|
||||
private void setHostAddrFromSparkConf(SparkConf sparkConf) {
|
||||
String hostAddr = sparkConf.get("spark.driver.host", null);
|
||||
if (hostAddr != null) {
|
||||
logger.info("Overriding hostIp to (" + hostAddr + ") found in spark-conf. It was " + this.hostAddr);
|
||||
this.hostAddr = hostAddr;
|
||||
} else {
|
||||
logger.warn("Unable to find driver bind address from spark config");
|
||||
}
|
||||
}
|
||||
|
||||
public void run() throws IOException {
|
||||
|
||||
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(timelineServer.getFs(), cfg.basePath, true);
|
||||
Collections.shuffle(allPartitionPaths);
|
||||
List<String> selected = allPartitionPaths.stream().filter(p -> !p.contains("error"))
|
||||
.limit(cfg.maxPartitions).collect(Collectors.toList());
|
||||
JavaSparkContext jsc = UtilHelpers.buildSparkContext("hudi-view-perf-" + cfg.basePath, cfg.sparkMaster);
|
||||
if (!useExternalTimelineServer) {
|
||||
this.timelineServer.startService();
|
||||
setHostAddrFromSparkConf(jsc.getConf());
|
||||
} else {
|
||||
this.hostAddr = cfg.serverHost;
|
||||
}
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(timelineServer.getConf(), cfg.basePath, true);
|
||||
SyncableFileSystemView fsView = new RemoteHoodieTableFileSystemView(this.hostAddr, cfg.serverPort, metaClient);
|
||||
|
||||
String reportDir = cfg.reportDir;
|
||||
metaClient.getFs().mkdirs(new Path(reportDir));
|
||||
|
||||
String dumpPrefix = UUID.randomUUID().toString();
|
||||
System.out.println("First Iteration to load all partitions");
|
||||
Dumper d = new Dumper(metaClient.getFs(), new Path(reportDir,
|
||||
String.format("1_%s.csv", dumpPrefix)));
|
||||
d.init();
|
||||
d.dump(runLookups(jsc, selected, fsView, 1, 0));
|
||||
d.close();
|
||||
System.out.println("\n\n\n First Iteration is done");
|
||||
|
||||
Dumper d2 = new Dumper(metaClient.getFs(), new Path(reportDir,
|
||||
String.format("2_%s.csv", dumpPrefix)));
|
||||
d2.init();
|
||||
d2.dump(runLookups(jsc, selected, fsView, cfg.numIterations, cfg.numCoresPerExecutor));
|
||||
d2.close();
|
||||
|
||||
System.out.println("\n\n\nDumping all File Slices");
|
||||
selected.stream().forEach(p -> fsView.getAllFileSlices(p).forEach(s -> System.out.println("\tMyFileSlice=" + s)));
|
||||
|
||||
// Waiting for curl queries
|
||||
if (!useExternalTimelineServer && cfg.waitForManualQueries) {
|
||||
System.out.println("Timeline Server Host Address=" + hostAddr + ", port=" + timelineServer.getServerPort());
|
||||
while (true) {
|
||||
try {
|
||||
Thread.sleep(60000);
|
||||
} catch (InterruptedException e) {
|
||||
// skip it
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public List<PerfStats> runLookups(JavaSparkContext jsc, List<String> partitionPaths, SyncableFileSystemView fsView,
|
||||
int numIterations, int concurrency) {
|
||||
List<PerfStats> perfStats = jsc.parallelize(partitionPaths, cfg.numExecutors).flatMap(p -> {
|
||||
ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(100);
|
||||
final List<PerfStats> result = new ArrayList<>();
|
||||
final List<ScheduledFuture<PerfStats>> futures = new ArrayList<>();
|
||||
List<FileSlice> slices = fsView.getLatestFileSlices(p).collect(Collectors.toList());
|
||||
String fileId = slices.isEmpty() ? "dummyId"
|
||||
: slices.get(new Random(Double.doubleToLongBits(Math.random())).nextInt(slices.size())).getFileId();
|
||||
IntStream.range(0, concurrency).forEach(i -> {
|
||||
futures.add(executor.schedule(() -> runOneRound(fsView, p, fileId, i, numIterations), 0, TimeUnit.NANOSECONDS));
|
||||
});
|
||||
futures.stream().forEach(x -> {
|
||||
try {
|
||||
result.add(x.get());
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
System.out.println("SLICES are=");
|
||||
slices.stream().forEach(s -> {
|
||||
System.out.println("\t\tFileSlice=" + s);
|
||||
});
|
||||
return result.iterator();
|
||||
}).collect();
|
||||
return perfStats;
|
||||
}
|
||||
|
||||
private static PerfStats runOneRound(SyncableFileSystemView fsView, String partition, String fileId, int id,
|
||||
int numIterations) {
|
||||
Histogram latencyHistogram = new Histogram(new UniformReservoir(10000));
|
||||
for (int i = 0; i < numIterations; i++) {
|
||||
long beginTs = System.currentTimeMillis();
|
||||
Option<FileSlice> c = fsView.getLatestFileSlice(partition, fileId);
|
||||
long endTs = System.currentTimeMillis();
|
||||
System.out.println("Latest File Slice for part=" + partition + ", fileId="
|
||||
+ fileId + ", Slice=" + c + ", Time=" + (endTs - beginTs));
|
||||
latencyHistogram.update(endTs - beginTs);
|
||||
}
|
||||
return new PerfStats(partition, id, latencyHistogram.getSnapshot());
|
||||
}
|
||||
|
||||
private static class Dumper implements Serializable {
|
||||
|
||||
private final Path dumpPath;
|
||||
private final FileSystem fileSystem;
|
||||
private FSDataOutputStream outputStream;
|
||||
|
||||
public Dumper(FileSystem fs, Path dumpPath) {
|
||||
this.dumpPath = dumpPath;
|
||||
this.fileSystem = fs;
|
||||
}
|
||||
|
||||
public void init() throws IOException {
|
||||
outputStream = fileSystem.create(dumpPath, true);
|
||||
addHeader();
|
||||
}
|
||||
|
||||
private void addHeader() throws IOException {
|
||||
String header = "Partition,Thread,Min,Max,Mean,Median,75th,95th\n";
|
||||
outputStream.write(header.getBytes());
|
||||
outputStream.flush();
|
||||
}
|
||||
|
||||
public void dump(List<PerfStats> stats) {
|
||||
stats.stream().forEach(x -> {
|
||||
String row = String.format("%s,%d,%d,%d,%f,%f,%f,%f\n", x.partition, x.id, x.minTime, x.maxTime, x.meanTime,
|
||||
x.medianTime, x.p75, x.p95);
|
||||
System.out.println(row);
|
||||
try {
|
||||
outputStream.write(row.getBytes());
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
outputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private static class PerfStats implements Serializable {
|
||||
|
||||
private final String partition;
|
||||
private final int id;
|
||||
private final long minTime;
|
||||
private final long maxTime;
|
||||
private final double meanTime;
|
||||
private final double medianTime;
|
||||
private final double p95;
|
||||
private final double p75;
|
||||
|
||||
public PerfStats(String partition, int id, Snapshot s) {
|
||||
this(partition, id, s.getMin(), s.getMax(), s.getMean(), s.getMedian(), s.get95thPercentile(),
|
||||
s.get75thPercentile());
|
||||
}
|
||||
|
||||
public PerfStats(String partition, int id, long minTime, long maxTime, double meanTime, double medianTime,
|
||||
double p95, double p75) {
|
||||
this.partition = partition;
|
||||
this.id = id;
|
||||
this.minTime = minTime;
|
||||
this.maxTime = maxTime;
|
||||
this.meanTime = meanTime;
|
||||
this.medianTime = medianTime;
|
||||
this.p95 = p95;
|
||||
this.p75 = p75;
|
||||
}
|
||||
}
|
||||
|
||||
public static class Config implements Serializable {
|
||||
|
||||
@Parameter(names = {"--base-path", "-b"}, description = "Base Path", required = true)
|
||||
public String basePath = "";
|
||||
|
||||
@Parameter(names = {"--report-dir", "-rd"}, description = "Dir where reports are added", required = true)
|
||||
public String reportDir = "";
|
||||
|
||||
@Parameter(names = {"--max-partitions", "-m"}, description = "Mx partitions to be loaded")
|
||||
public Integer maxPartitions = 1000;
|
||||
|
||||
@Parameter(names = {"--num-executors", "-e"}, description = "num executors")
|
||||
public Integer numExecutors = 10;
|
||||
|
||||
@Parameter(names = {"--num-cores", "-c"}, description = "num cores")
|
||||
public Integer numCoresPerExecutor = 10;
|
||||
|
||||
@Parameter(names = {"--num-iterations", "-i"}, description = "Number of iterations for each partitions")
|
||||
public Integer numIterations = 10;
|
||||
|
||||
@Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false)
|
||||
public String sparkMaster = "local[2]";
|
||||
|
||||
@Parameter(names = {"--server-port", "-p"}, description = " Server Port")
|
||||
public Integer serverPort = 26754;
|
||||
|
||||
@Parameter(names = {"--server-host", "-sh"},
|
||||
description = " Server Host (Set it for externally managed timeline service")
|
||||
public String serverHost = null;
|
||||
|
||||
@Parameter(names = {"--view-storage", "-st"}, description = "View Storage Type. Defaut - SPILLABLE_DISK")
|
||||
public FileSystemViewStorageType viewStorageType = FileSystemViewStorageType.SPILLABLE_DISK;
|
||||
|
||||
@Parameter(names = {"--max-view-mem-per-table", "-mv"},
|
||||
description = "Maximum view memory per table in MB to be used for storing file-groups."
|
||||
+ " Overflow file-groups will be spilled to disk. Used for SPILLABLE_DISK storage type")
|
||||
public Integer maxViewMemPerTableInMB = 2048;
|
||||
|
||||
@Parameter(names = {"--mem-overhead-fraction-pending-compaction", "-cf"},
|
||||
description = "Memory Fraction of --max-view-mem-per-table to be allocated for managing pending compaction"
|
||||
+ " storage. Overflow entries will be spilled to disk. Used for SPILLABLE_DISK storage type")
|
||||
public Double memFractionForCompactionPerTable = 0.001;
|
||||
|
||||
@Parameter(names = {"--base-store-path", "-sp"},
|
||||
description = "Directory where spilled view entries will be stored. Used for SPILLABLE_DISK storage type")
|
||||
public String baseStorePathForFileGroups = FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR;
|
||||
|
||||
@Parameter(names = {"--rocksdb-path", "-rp"},
|
||||
description = "Root directory for RocksDB")
|
||||
public String rocksDBPath = FileSystemViewStorageConfig.DEFAULT_ROCKSDB_BASE_PATH;
|
||||
|
||||
@Parameter(names = {"--wait-for-manual-queries", "-ww"})
|
||||
public Boolean waitForManualQueries = false;
|
||||
|
||||
@Parameter(names = {"--help", "-h"})
|
||||
public Boolean help = false;
|
||||
|
||||
public TimelineService.Config getTimelinServerConfig() {
|
||||
TimelineService.Config c = new TimelineService.Config();
|
||||
c.viewStorageType = viewStorageType;
|
||||
c.baseStorePathForFileGroups = baseStorePathForFileGroups;
|
||||
c.maxViewMemPerTableInMB = maxViewMemPerTableInMB;
|
||||
c.memFractionForCompactionPerTable = memFractionForCompactionPerTable;
|
||||
c.rocksDBPath = rocksDBPath;
|
||||
c.serverPort = serverPort;
|
||||
return c;
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
final Config cfg = new Config();
|
||||
JCommander cmd = new JCommander(cfg, args);
|
||||
if (cfg.help || args.length == 0) {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
TimelineServerPerf perf = new TimelineServerPerf(cfg);
|
||||
perf.run();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,82 @@
|
||||
/*
|
||||
* 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.utilities.schema;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* A simple schema provider, that reads off files on DFS
|
||||
*/
|
||||
public class FilebasedSchemaProvider extends SchemaProvider {
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
public static class Config {
|
||||
private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider"
|
||||
+ ".source.schema.file";
|
||||
private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider"
|
||||
+ ".target.schema.file";
|
||||
}
|
||||
|
||||
private final FileSystem fs;
|
||||
|
||||
private final Schema sourceSchema;
|
||||
|
||||
private Schema targetSchema;
|
||||
|
||||
public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||
super(props, jssc);
|
||||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP));
|
||||
this.fs = FSUtils.getFs(props.getString(Config.SOURCE_SCHEMA_FILE_PROP), jssc.hadoopConfiguration());
|
||||
try {
|
||||
this.sourceSchema = new Schema.Parser().parse(
|
||||
fs.open(new Path(props.getString(Config.SOURCE_SCHEMA_FILE_PROP))));
|
||||
if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) {
|
||||
this.targetSchema = new Schema.Parser().parse(
|
||||
fs.open(new Path(props.getString(Config.TARGET_SCHEMA_FILE_PROP))));
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading schema", ioe);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return sourceSchema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getTargetSchema() {
|
||||
if (targetSchema != null) {
|
||||
return targetSchema;
|
||||
} else {
|
||||
return super.getTargetSchema();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.utilities.schema;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.AvroConversionUtils;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
public class RowBasedSchemaProvider extends SchemaProvider {
|
||||
|
||||
// Used in GenericRecord conversions
|
||||
public static final String HOODIE_RECORD_NAMESPACE = "hoodie.source";
|
||||
public static final String HOODIE_RECORD_STRUCT_NAME = "hoodie_source";
|
||||
|
||||
private StructType rowStruct;
|
||||
|
||||
public RowBasedSchemaProvider(StructType rowStruct) {
|
||||
super(null, null);
|
||||
this.rowStruct = rowStruct;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return AvroConversionUtils.convertStructTypeToAvroSchema(rowStruct, HOODIE_RECORD_STRUCT_NAME,
|
||||
HOODIE_RECORD_NAMESPACE);
|
||||
}
|
||||
}
|
||||
@@ -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.hudi.utilities.schema;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Class to provide schema for reading data and also writing into a Hoodie table
|
||||
*/
|
||||
public abstract class SchemaProvider implements Serializable {
|
||||
|
||||
protected TypedProperties config;
|
||||
|
||||
protected JavaSparkContext jssc;
|
||||
|
||||
protected SchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||
this.config = props;
|
||||
this.jssc = jssc;
|
||||
}
|
||||
|
||||
public abstract Schema getSourceSchema();
|
||||
|
||||
public Schema getTargetSchema() {
|
||||
// by default, use source schema as target for hoodie dataset as well
|
||||
return getSourceSchema();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,89 @@
|
||||
/*
|
||||
* 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.utilities.schema;
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.Collections;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Obtains latest schema from the Confluent/Kafka schema-registry
|
||||
*
|
||||
* https://github.com/confluentinc/schema-registry
|
||||
*/
|
||||
public class SchemaRegistryProvider extends SchemaProvider {
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
public static class Config {
|
||||
|
||||
private static final String SRC_SCHEMA_REGISTRY_URL_PROP = "hoodie.deltastreamer.schemaprovider.registry.url";
|
||||
private static final String TARGET_SCHEMA_REGISTRY_URL_PROP =
|
||||
"hoodie.deltastreamer.schemaprovider.registry.targetUrl";
|
||||
}
|
||||
|
||||
private final Schema schema;
|
||||
private final Schema targetSchema;
|
||||
|
||||
private static String fetchSchemaFromRegistry(String registryUrl) throws IOException {
|
||||
URL registry = new URL(registryUrl);
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
JsonNode node = mapper.readTree(registry.openStream());
|
||||
return node.get("schema").asText();
|
||||
}
|
||||
|
||||
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||
super(props, jssc);
|
||||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
|
||||
String registryUrl = props.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
|
||||
String targetRegistryUrl = props.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);
|
||||
try {
|
||||
this.schema = getSchema(registryUrl);
|
||||
if (!targetRegistryUrl.equals(registryUrl)) {
|
||||
this.targetSchema = getSchema(targetRegistryUrl);
|
||||
} else {
|
||||
this.targetSchema = schema;
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException("Error reading schema from registry :" + registryUrl, ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private static Schema getSchema(String registryUrl) throws IOException {
|
||||
return new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return schema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getTargetSchema() {
|
||||
return targetSchema;
|
||||
}
|
||||
}
|
||||
@@ -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.hudi.utilities.sources;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.mapred.AvroKey;
|
||||
import org.apache.avro.mapreduce.AvroKeyInputFormat;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.helpers.DFSPathSelector;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
/**
|
||||
* DFS Source that reads avro data
|
||||
*/
|
||||
public class AvroDFSSource extends AvroSource {
|
||||
|
||||
private final DFSPathSelector pathSelector;
|
||||
|
||||
public AvroDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Option<String> lastCkptStr,
|
||||
long sourceLimit) {
|
||||
Pair<Option<String>, String> selectPathsWithMaxModificationTime =
|
||||
pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit);
|
||||
return selectPathsWithMaxModificationTime.getLeft().map(pathStr -> new InputBatch<>(
|
||||
Option.of(fromFiles(pathStr)),
|
||||
selectPathsWithMaxModificationTime.getRight()))
|
||||
.orElseGet(() -> new InputBatch<>(Option.empty(), selectPathsWithMaxModificationTime.getRight()));
|
||||
}
|
||||
|
||||
private JavaRDD<GenericRecord> fromFiles(String pathStr) {
|
||||
JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr,
|
||||
AvroKeyInputFormat.class, AvroKey.class, NullWritable.class,
|
||||
sparkContext.hadoopConfiguration());
|
||||
return avroRDD.keys().map(r -> ((GenericRecord) r.datum()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import io.confluent.kafka.serializers.KafkaAvroDecoder;
|
||||
import kafka.serializer.StringDecoder;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
|
||||
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.streaming.kafka.KafkaUtils;
|
||||
import org.apache.spark.streaming.kafka.OffsetRange;
|
||||
|
||||
/**
|
||||
* Reads avro serialized Kafka data, based on the confluent schema-registry
|
||||
*/
|
||||
public class AvroKafkaSource extends AvroSource {
|
||||
|
||||
private static Logger log = LogManager.getLogger(AvroKafkaSource.class);
|
||||
|
||||
private final KafkaOffsetGen offsetGen;
|
||||
|
||||
public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
offsetGen = new KafkaOffsetGen(props);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Option<String> lastCheckpointStr,
|
||||
long sourceLimit) {
|
||||
OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit);
|
||||
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
|
||||
if (totalNewMsgs <= 0) {
|
||||
return new InputBatch<>(Option.empty(),
|
||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
||||
} else {
|
||||
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName());
|
||||
}
|
||||
JavaRDD<GenericRecord> newDataRDD = toRDD(offsetRanges);
|
||||
return new InputBatch<>(Option.of(newDataRDD),
|
||||
KafkaOffsetGen.CheckpointUtils.offsetsToStr(offsetRanges));
|
||||
}
|
||||
|
||||
private JavaRDD<GenericRecord> toRDD(OffsetRange[] offsetRanges) {
|
||||
JavaRDD<GenericRecord> recordRDD = KafkaUtils
|
||||
.createRDD(sparkContext, String.class, Object.class, StringDecoder.class, KafkaAvroDecoder.class,
|
||||
offsetGen.getKafkaParams(), offsetRanges).values().map(obj -> (GenericRecord) obj);
|
||||
return recordRDD;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
public abstract class AvroSource extends Source<JavaRDD<GenericRecord>> {
|
||||
|
||||
public AvroSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext,
|
||||
SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider, SourceType.AVRO);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.mapred.AvroKey;
|
||||
import org.apache.avro.mapreduce.AvroKeyInputFormat;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.NullWritable;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.utilities.HiveIncrementalPuller;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
/**
|
||||
* Source to read deltas produced by {@link HiveIncrementalPuller}, commit by commit and apply
|
||||
* to the target table
|
||||
* <p>
|
||||
* The general idea here is to have commits sync across the data pipeline.
|
||||
* <p>
|
||||
* [Source Tables(s)] ====> HiveIncrementalScanner ==> incrPullRootPath ==> targetTable {c1,c2,c3,...}
|
||||
* {c1,c2,c3,...} {c1,c2,c3,...}
|
||||
* <p>
|
||||
* This produces beautiful causality, that makes data issues in ETLs very easy to debug
|
||||
*/
|
||||
public class HiveIncrPullSource extends AvroSource {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(HiveIncrPullSource.class);
|
||||
|
||||
private final transient FileSystem fs;
|
||||
|
||||
private final String incrPullRootPath;
|
||||
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.incrpull.root";
|
||||
}
|
||||
|
||||
public HiveIncrPullSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.ROOT_INPUT_PATH_PROP));
|
||||
this.incrPullRootPath = props.getString(Config.ROOT_INPUT_PATH_PROP);
|
||||
this.fs = FSUtils.getFs(incrPullRootPath, sparkContext.hadoopConfiguration());
|
||||
}
|
||||
|
||||
/**
|
||||
* Finds the first commit from source, greater than the target's last commit, and reads it out.
|
||||
*/
|
||||
private Option<String> findCommitToPull(Option<String> latestTargetCommit)
|
||||
throws IOException {
|
||||
|
||||
log.info("Looking for commits ");
|
||||
|
||||
FileStatus[] commitTimePaths = fs.listStatus(new Path(incrPullRootPath));
|
||||
List<String> commitTimes = new ArrayList<>(commitTimePaths.length);
|
||||
for (FileStatus commitTimePath : commitTimePaths) {
|
||||
String[] splits = commitTimePath.getPath().toString().split("/");
|
||||
commitTimes.add(splits[splits.length - 1]);
|
||||
}
|
||||
Collections.sort(commitTimes);
|
||||
log.info("Retrieved commit times " + commitTimes);
|
||||
|
||||
if (!latestTargetCommit.isPresent()) {
|
||||
// start from the beginning
|
||||
return Option.of(commitTimes.get(0));
|
||||
}
|
||||
|
||||
for (String commitTime : commitTimes) {
|
||||
//TODO(vc): Add an option to delete consumed commits
|
||||
if (commitTime.compareTo(latestTargetCommit.get()) > 0) {
|
||||
return Option.of(commitTime);
|
||||
}
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(
|
||||
Option<String> lastCheckpointStr, long sourceLimit) {
|
||||
try {
|
||||
// find the source commit to pull
|
||||
Option<String> commitToPull = findCommitToPull(lastCheckpointStr);
|
||||
|
||||
if (!commitToPull.isPresent()) {
|
||||
return new InputBatch<>(Option.empty(),
|
||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
||||
}
|
||||
|
||||
// read the files out.
|
||||
List<FileStatus> commitDeltaFiles = Arrays.asList(
|
||||
fs.listStatus(new Path(incrPullRootPath, commitToPull.get())));
|
||||
String pathStr = commitDeltaFiles.stream().map(f -> f.getPath().toString())
|
||||
.collect(Collectors.joining(","));
|
||||
JavaPairRDD<AvroKey, NullWritable> avroRDD = sparkContext.newAPIHadoopFile(pathStr,
|
||||
AvroKeyInputFormat.class, AvroKey.class, NullWritable.class,
|
||||
sparkContext.hadoopConfiguration());
|
||||
return new InputBatch<>(Option.of(avroRDD.keys().map(r -> ((GenericRecord) r.datum()))),
|
||||
String.valueOf(commitToPull.get()));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(
|
||||
"Unable to read from source from checkpoint: " + lastCheckpointStr, ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,158 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import java.util.Arrays;
|
||||
import org.apache.hudi.DataSourceReadOptions;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.helpers.IncrSourceHelper;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.DataFrameReader;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
public class HoodieIncrSource extends RowSource {
|
||||
protected static class Config {
|
||||
|
||||
/**
|
||||
* {@value #HOODIE_SRC_BASE_PATH} is the base-path for the source Hoodie table
|
||||
*/
|
||||
private static final String HOODIE_SRC_BASE_PATH = "hoodie.deltastreamer.source.hoodieincr.path";
|
||||
|
||||
/**
|
||||
* {@value #NUM_INSTANTS_PER_FETCH} allows the max number of instants whose changes can be incrementally fetched
|
||||
*/
|
||||
private static final String NUM_INSTANTS_PER_FETCH = "hoodie.deltastreamer.source.hoodieincr.num_instants";
|
||||
private static final Integer DEFAULT_NUM_INSTANTS_PER_FETCH = 1;
|
||||
|
||||
/**
|
||||
* {@value #HOODIE_SRC_PARTITION_FIELDS} specifies partition fields that needs to be added to source table after
|
||||
* parsing _hoodie_partition_path
|
||||
*/
|
||||
private static final String HOODIE_SRC_PARTITION_FIELDS = "hoodie.deltastreamer.source.hoodieincr.partition.fields";
|
||||
|
||||
/**
|
||||
* {@value #HOODIE_SRC_PARTITION_EXTRACTORCLASS} PartitionValueExtractor class to extract partition fields from
|
||||
* _hoodie_partition_path
|
||||
*/
|
||||
private static final String HOODIE_SRC_PARTITION_EXTRACTORCLASS =
|
||||
"hoodie.deltastreamer.source.hoodieincr.partition.extractor.class";
|
||||
private static final String DEFAULT_HOODIE_SRC_PARTITION_EXTRACTORCLASS =
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName();
|
||||
|
||||
/**
|
||||
* {@value #READ_LATEST_INSTANT_ON_MISSING_CKPT} allows delta-streamer to incrementally fetch from latest committed
|
||||
* instant when checkpoint is not provided.
|
||||
*/
|
||||
private static final String READ_LATEST_INSTANT_ON_MISSING_CKPT =
|
||||
"hoodie.deltastreamer.source.hoodieincr.read_latest_on_missing_ckpt";
|
||||
private static final Boolean DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT = false;
|
||||
}
|
||||
|
||||
public HoodieIncrSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr, long sourceLimit) {
|
||||
|
||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH));
|
||||
|
||||
/**
|
||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH,
|
||||
Config.HOODIE_SRC_PARTITION_FIELDS));
|
||||
List<String> partitionFields = props.getStringList(Config.HOODIE_SRC_PARTITION_FIELDS, ",",
|
||||
new ArrayList<>());
|
||||
PartitionValueExtractor extractor = DataSourceUtils.createPartitionExtractor(props.getString(
|
||||
Config.HOODIE_SRC_PARTITION_EXTRACTORCLASS, Config.DEFAULT_HOODIE_SRC_PARTITION_EXTRACTORCLASS));
|
||||
**/
|
||||
String srcPath = props.getString(Config.HOODIE_SRC_BASE_PATH);
|
||||
int numInstantsPerFetch = props.getInteger(Config.NUM_INSTANTS_PER_FETCH, Config.DEFAULT_NUM_INSTANTS_PER_FETCH);
|
||||
boolean readLatestOnMissingCkpt = props.getBoolean(Config.READ_LATEST_INSTANT_ON_MISSING_CKPT,
|
||||
Config.DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT);
|
||||
|
||||
// Use begin Instant if set and non-empty
|
||||
Option<String> beginInstant =
|
||||
lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() : lastCkptStr : Option.empty();
|
||||
|
||||
Pair<String, String> instantEndpts = IncrSourceHelper.calculateBeginAndEndInstants(sparkContext, srcPath,
|
||||
numInstantsPerFetch, beginInstant, readLatestOnMissingCkpt);
|
||||
|
||||
if (instantEndpts.getKey().equals(instantEndpts.getValue())) {
|
||||
log.warn("Already caught up. Begin Checkpoint was :" + instantEndpts.getKey());
|
||||
return Pair.of(Option.empty(), instantEndpts.getKey());
|
||||
}
|
||||
|
||||
// Do Incr pull. Set end instant if available
|
||||
DataFrameReader reader = sparkSession.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), instantEndpts.getLeft())
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY(), instantEndpts.getRight());
|
||||
|
||||
Dataset<Row> source = reader.load(srcPath);
|
||||
|
||||
/**
|
||||
log.info("Partition Fields are : (" + partitionFields + "). Initial Source Schema :" + source.schema());
|
||||
|
||||
StructType newSchema = new StructType(source.schema().fields());
|
||||
for (String field : partitionFields) {
|
||||
newSchema = newSchema.add(field, DataTypes.StringType, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates if the commit time is sane and also generates Partition fields from _hoodie_partition_path if
|
||||
* configured
|
||||
*
|
||||
Dataset<Row> validated = source.map((MapFunction<Row, Row>) (Row row) -> {
|
||||
// _hoodie_instant_time
|
||||
String instantTime = row.getString(0);
|
||||
IncrSourceHelper.validateInstantTime(row, instantTime, instantEndpts.getKey(), instantEndpts.getValue());
|
||||
if (!partitionFields.isEmpty()) {
|
||||
// _hoodie_partition_path
|
||||
String hoodiePartitionPath = row.getString(3);
|
||||
List<Object> partitionVals = extractor.extractPartitionValuesInPath(hoodiePartitionPath).stream()
|
||||
.map(o -> (Object) o).collect(Collectors.toList());
|
||||
Preconditions.checkArgument(partitionVals.size() == partitionFields.size(),
|
||||
"#partition-fields != #partition-values-extracted");
|
||||
List<Object> rowObjs = new ArrayList<>(scala.collection.JavaConversions.seqAsJavaList(row.toSeq()));
|
||||
rowObjs.addAll(partitionVals);
|
||||
return RowFactory.create(rowObjs.toArray());
|
||||
}
|
||||
return row;
|
||||
}, RowEncoder.apply(newSchema));
|
||||
|
||||
log.info("Validated Source Schema :" + validated.schema());
|
||||
**/
|
||||
|
||||
// Remove Hoodie meta columns except partition path from input source
|
||||
final Dataset<Row> src = source.drop(HoodieRecord.HOODIE_META_COLUMNS.stream()
|
||||
.filter(x -> !x.equals(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toArray(String[]::new));
|
||||
//log.info("Final Schema from Source is :" + src.schema());
|
||||
return Pair.of(Option.of(src), instantEndpts.getRight());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
|
||||
public class InputBatch<T> {
|
||||
|
||||
private final Option<T> batch;
|
||||
private final String checkpointForNextBatch;
|
||||
private final SchemaProvider schemaProvider;
|
||||
|
||||
public InputBatch(Option<T> batch, String checkpointForNextBatch,
|
||||
SchemaProvider schemaProvider) {
|
||||
this.batch = batch;
|
||||
this.checkpointForNextBatch = checkpointForNextBatch;
|
||||
this.schemaProvider = schemaProvider;
|
||||
}
|
||||
|
||||
public InputBatch(Option<T> batch, String checkpointForNextBatch) {
|
||||
this.batch = batch;
|
||||
this.checkpointForNextBatch = checkpointForNextBatch;
|
||||
this.schemaProvider = null;
|
||||
}
|
||||
|
||||
public Option<T> getBatch() {
|
||||
return batch;
|
||||
}
|
||||
|
||||
public String getCheckpointForNextBatch() {
|
||||
return checkpointForNextBatch;
|
||||
}
|
||||
|
||||
public SchemaProvider getSchemaProvider() {
|
||||
return schemaProvider;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.helpers.DFSPathSelector;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
/**
|
||||
* DFS Source that reads json data
|
||||
*/
|
||||
public class JsonDFSSource extends JsonSource {
|
||||
|
||||
private final DFSPathSelector pathSelector;
|
||||
|
||||
public JsonDFSSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
this.pathSelector = new DFSPathSelector(props, sparkContext.hadoopConfiguration());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<String>> fetchNewData(Option<String> lastCkptStr,
|
||||
long sourceLimit) {
|
||||
Pair<Option<String>, String> selPathsWithMaxModificationTime =
|
||||
pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit);
|
||||
return selPathsWithMaxModificationTime.getLeft().map(pathStr -> new InputBatch<>(
|
||||
Option.of(fromFiles(pathStr)), selPathsWithMaxModificationTime.getRight()))
|
||||
.orElse(new InputBatch<>(Option.empty(), selPathsWithMaxModificationTime.getRight()));
|
||||
}
|
||||
|
||||
private JavaRDD<String> fromFiles(String pathStr) {
|
||||
return sparkContext.textFile(pathStr);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import kafka.serializer.StringDecoder;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
|
||||
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.apache.spark.streaming.kafka.KafkaUtils;
|
||||
import org.apache.spark.streaming.kafka.OffsetRange;
|
||||
|
||||
/**
|
||||
* Read json kafka data
|
||||
*/
|
||||
public class JsonKafkaSource extends JsonSource {
|
||||
|
||||
private static Logger log = LogManager.getLogger(JsonKafkaSource.class);
|
||||
|
||||
private final KafkaOffsetGen offsetGen;
|
||||
|
||||
public JsonKafkaSource(TypedProperties properties, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(properties, sparkContext, sparkSession, schemaProvider);
|
||||
offsetGen = new KafkaOffsetGen(properties);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputBatch<JavaRDD<String>> fetchNewData(Option<String> lastCheckpointStr,
|
||||
long sourceLimit) {
|
||||
OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit);
|
||||
long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges);
|
||||
if (totalNewMsgs <= 0) {
|
||||
return new InputBatch<>(Option.empty(),
|
||||
lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : "");
|
||||
}
|
||||
log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName());
|
||||
JavaRDD<String> newDataRDD = toRDD(offsetRanges);
|
||||
return new InputBatch<>(Option.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges));
|
||||
}
|
||||
|
||||
private JavaRDD<String> toRDD(OffsetRange[] offsetRanges) {
|
||||
return KafkaUtils.createRDD(sparkContext, String.class, String.class, StringDecoder.class, StringDecoder.class,
|
||||
offsetGen.getKafkaParams(), offsetRanges).values();
|
||||
}
|
||||
}
|
||||
@@ -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.utilities.sources;
|
||||
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
public abstract class JsonSource extends Source<JavaRDD<String>> {
|
||||
|
||||
public JsonSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext,
|
||||
SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider, SourceType.JSON);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
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;
|
||||
|
||||
public abstract class RowSource extends Source<Dataset<Row>> {
|
||||
|
||||
public RowSource(TypedProperties props,
|
||||
JavaSparkContext sparkContext,
|
||||
SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider, SourceType.ROW);
|
||||
}
|
||||
|
||||
protected abstract Pair<Option<Dataset<Row>>, String> fetchNextBatch(Option<String> lastCkptStr,
|
||||
long sourceLimit);
|
||||
|
||||
@Override
|
||||
protected final InputBatch<Dataset<Row>> fetchNewData(Option<String> lastCkptStr, long sourceLimit) {
|
||||
Pair<Option<Dataset<Row>>, String> res = fetchNextBatch(lastCkptStr, sourceLimit);
|
||||
return res.getKey().map(dsr -> {
|
||||
SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(dsr.schema());
|
||||
return new InputBatch<>(res.getKey(), res.getValue(), rowSchemaProvider);
|
||||
}).orElseGet(() -> new InputBatch<>(res.getKey(), res.getValue()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,85 @@
|
||||
/*
|
||||
* 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.utilities.sources;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
/**
|
||||
* Represents a source from which we can tail data. Assumes a constructor that takes properties.
|
||||
*/
|
||||
public abstract class Source<T> implements Serializable {
|
||||
protected static volatile Logger log = LogManager.getLogger(Source.class);
|
||||
|
||||
public enum SourceType {
|
||||
JSON,
|
||||
AVRO,
|
||||
ROW
|
||||
}
|
||||
|
||||
protected transient TypedProperties props;
|
||||
protected transient JavaSparkContext sparkContext;
|
||||
protected transient SparkSession sparkSession;
|
||||
private transient SchemaProvider overriddenSchemaProvider;
|
||||
|
||||
private final SourceType sourceType;
|
||||
|
||||
protected Source(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider) {
|
||||
this(props, sparkContext, sparkSession, schemaProvider, SourceType.AVRO);
|
||||
}
|
||||
|
||||
protected Source(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider, SourceType sourceType) {
|
||||
this.props = props;
|
||||
this.sparkContext = sparkContext;
|
||||
this.sparkSession = sparkSession;
|
||||
this.overriddenSchemaProvider = schemaProvider;
|
||||
this.sourceType = sourceType;
|
||||
}
|
||||
|
||||
protected abstract InputBatch<T> fetchNewData(Option<String> lastCkptStr, long sourceLimit);
|
||||
|
||||
/**
|
||||
* Main API called by Hoodie Delta Streamer to fetch records
|
||||
* @param lastCkptStr Last Checkpoint
|
||||
* @param sourceLimit Source Limit
|
||||
* @return
|
||||
*/
|
||||
public final InputBatch<T> fetchNext(Option<String> lastCkptStr, long sourceLimit) {
|
||||
InputBatch<T> batch = fetchNewData(lastCkptStr, sourceLimit);
|
||||
// If overriddenSchemaProvider is passed in CLI, use it
|
||||
return overriddenSchemaProvider == null ? batch : new InputBatch<>(batch.getBatch(),
|
||||
batch.getCheckpointForNextBatch(), overriddenSchemaProvider);
|
||||
}
|
||||
|
||||
public SourceType getSourceType() {
|
||||
return sourceType;
|
||||
}
|
||||
|
||||
public SparkSession getSparkSession() {
|
||||
return sparkSession;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,99 @@
|
||||
/*
|
||||
* 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.utilities.sources.helpers;
|
||||
|
||||
import com.twitter.bijection.Injection;
|
||||
import com.twitter.bijection.avro.GenericAvroCodecs;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hudi.avro.MercifulJsonConverter;
|
||||
|
||||
/**
|
||||
* Convert a variety of datum into Avro GenericRecords. Has a bunch of lazy
|
||||
* fields to circumvent issues around serializing these objects from driver to executors
|
||||
*/
|
||||
public class AvroConvertor implements Serializable {
|
||||
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient Schema schema;
|
||||
|
||||
private final String schemaStr;
|
||||
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient MercifulJsonConverter jsonConverter;
|
||||
|
||||
|
||||
/**
|
||||
* To be lazily inited on executors
|
||||
*/
|
||||
private transient Injection<GenericRecord, byte[]> recordInjection;
|
||||
|
||||
|
||||
public AvroConvertor(String schemaStr) {
|
||||
this.schemaStr = schemaStr;
|
||||
}
|
||||
|
||||
public AvroConvertor(Schema schema) {
|
||||
this.schemaStr = schema.toString();
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
private void initSchema() {
|
||||
if (schema == null) {
|
||||
Schema.Parser parser = new Schema.Parser();
|
||||
schema = parser.parse(schemaStr);
|
||||
}
|
||||
}
|
||||
|
||||
private void initInjection() {
|
||||
if (recordInjection == null) {
|
||||
recordInjection = GenericAvroCodecs.toBinary(schema);
|
||||
}
|
||||
}
|
||||
|
||||
private void initJsonConvertor() {
|
||||
if (jsonConverter == null) {
|
||||
jsonConverter = new MercifulJsonConverter(schema);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public GenericRecord fromJson(String json) throws IOException {
|
||||
initSchema();
|
||||
initJsonConvertor();
|
||||
return jsonConverter.convert(json);
|
||||
}
|
||||
|
||||
public Schema getSchema() {
|
||||
return new Schema.Parser().parse(schemaStr);
|
||||
}
|
||||
|
||||
|
||||
public GenericRecord fromAvroBinary(byte[] avroBinary) {
|
||||
initSchema();
|
||||
initInjection();
|
||||
return recordInjection.invert(avroBinary).get();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,120 @@
|
||||
/*
|
||||
* 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.utilities.sources.helpers;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
public class DFSPathSelector {
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private static final String ROOT_INPUT_PATH_PROP = "hoodie.deltastreamer.source.dfs.root";
|
||||
}
|
||||
|
||||
private static final List<String> IGNORE_FILEPREFIX_LIST = Arrays.asList(".", "_");
|
||||
|
||||
private final transient FileSystem fs;
|
||||
private final TypedProperties props;
|
||||
|
||||
public DFSPathSelector(TypedProperties props, Configuration hadoopConf) {
|
||||
DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.ROOT_INPUT_PATH_PROP));
|
||||
this.props = props;
|
||||
this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), hadoopConf);
|
||||
}
|
||||
|
||||
public Pair<Option<String>, String> getNextFilePathsAndMaxModificationTime(
|
||||
Option<String> lastCheckpointStr, long sourceLimit) {
|
||||
|
||||
try {
|
||||
// obtain all eligible files under root folder.
|
||||
List<FileStatus> eligibleFiles = new ArrayList<>();
|
||||
RemoteIterator<LocatedFileStatus> fitr = fs.listFiles(
|
||||
new Path(props.getString(Config.ROOT_INPUT_PATH_PROP)), true);
|
||||
while (fitr.hasNext()) {
|
||||
LocatedFileStatus fileStatus = fitr.next();
|
||||
if (fileStatus.isDirectory() || IGNORE_FILEPREFIX_LIST.stream()
|
||||
.anyMatch(pfx -> fileStatus.getPath().getName().startsWith(pfx))) {
|
||||
continue;
|
||||
}
|
||||
eligibleFiles.add(fileStatus);
|
||||
}
|
||||
// sort them by modification time.
|
||||
eligibleFiles.sort(Comparator.comparingLong(FileStatus::getModificationTime));
|
||||
|
||||
// Filter based on checkpoint & input size, if needed
|
||||
long currentBytes = 0;
|
||||
long maxModificationTime = Long.MIN_VALUE;
|
||||
List<FileStatus> filteredFiles = new ArrayList<>();
|
||||
for (FileStatus f : eligibleFiles) {
|
||||
if (lastCheckpointStr.isPresent() && f.getModificationTime() <= Long.valueOf(
|
||||
lastCheckpointStr.get())) {
|
||||
// skip processed files
|
||||
continue;
|
||||
}
|
||||
|
||||
if (currentBytes + f.getLen() >= sourceLimit) {
|
||||
// we have enough data, we are done
|
||||
break;
|
||||
}
|
||||
|
||||
maxModificationTime = f.getModificationTime();
|
||||
currentBytes += f.getLen();
|
||||
filteredFiles.add(f);
|
||||
}
|
||||
|
||||
// no data to read
|
||||
if (filteredFiles.size() == 0) {
|
||||
return new ImmutablePair<>(Option.empty(),
|
||||
lastCheckpointStr.orElseGet(() -> String.valueOf(Long.MIN_VALUE)));
|
||||
}
|
||||
|
||||
// read the files out.
|
||||
String pathStr = filteredFiles.stream().map(f -> f.getPath().toString())
|
||||
.collect(Collectors.joining(","));
|
||||
|
||||
return new ImmutablePair<>(
|
||||
Option.ofNullable(pathStr),
|
||||
String.valueOf(maxModificationTime));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(
|
||||
"Unable to read from source from checkpoint: " + lastCheckpointStr, ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,103 @@
|
||||
/*
|
||||
* 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.utilities.sources.helpers;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Row;
|
||||
|
||||
public class IncrSourceHelper {
|
||||
|
||||
/**
|
||||
* Get a timestamp which is the next value in a descending sequence
|
||||
*
|
||||
* @param timestamp Timestamp
|
||||
*/
|
||||
private static String getStrictlyLowerTimestamp(String timestamp) {
|
||||
long ts = Long.parseLong(timestamp);
|
||||
Preconditions.checkArgument(ts > 0, "Timestamp must be positive");
|
||||
Long lower = ts - 1;
|
||||
return "" + lower;
|
||||
}
|
||||
|
||||
/**
|
||||
* Find begin and end instants to be set for the next fetch
|
||||
*
|
||||
* @param jssc Java Spark Context
|
||||
* @param srcBasePath Base path of Hudi source table
|
||||
* @param numInstantsPerFetch Max Instants per fetch
|
||||
* @param beginInstant Last Checkpoint String
|
||||
* @param readLatestOnMissingBeginInstant when begin instant is missing, allow reading from latest committed instant
|
||||
* @return begin and end instants
|
||||
*/
|
||||
public static Pair<String, String> calculateBeginAndEndInstants(
|
||||
JavaSparkContext jssc, String srcBasePath, int numInstantsPerFetch, Option<String> beginInstant,
|
||||
boolean readLatestOnMissingBeginInstant) {
|
||||
Preconditions.checkArgument(numInstantsPerFetch > 0, "Make sure the config"
|
||||
+ " hoodie.deltastreamer.source.hoodieincr.num_instants is set to a positive value");
|
||||
HoodieTableMetaClient srcMetaClient = new HoodieTableMetaClient(jssc.hadoopConfiguration(),
|
||||
srcBasePath, true);
|
||||
|
||||
final HoodieTimeline activeCommitTimeline =
|
||||
srcMetaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
|
||||
String beginInstantTime = beginInstant.orElseGet(() -> {
|
||||
if (readLatestOnMissingBeginInstant) {
|
||||
Option<HoodieInstant> lastInstant = activeCommitTimeline.lastInstant();
|
||||
return lastInstant.map(hoodieInstant -> getStrictlyLowerTimestamp(hoodieInstant.getTimestamp())).orElse("000");
|
||||
} else {
|
||||
throw new IllegalArgumentException("Missing begin instant for incremental pull. For reading from latest "
|
||||
+ "committed instant set hoodie.deltastreamer.source.hoodie.read_latest_on_midding_ckpt to true");
|
||||
}
|
||||
});
|
||||
|
||||
Option<HoodieInstant> nthInstant = Option.fromJavaOptional(
|
||||
activeCommitTimeline
|
||||
.findInstantsAfter(beginInstantTime, numInstantsPerFetch)
|
||||
.getInstants()
|
||||
.reduce((x, y) -> y));
|
||||
return Pair.of(beginInstantTime, nthInstant.map(instant -> instant.getTimestamp()).orElse(beginInstantTime));
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate instant time seen in the incoming row
|
||||
*
|
||||
* @param row Input Row
|
||||
* @param instantTime Hoodie Instant time of the row
|
||||
* @param sinceInstant begin instant of the batch
|
||||
* @param endInstant end instant of the batch
|
||||
*/
|
||||
public static void validateInstantTime(Row row, String instantTime, String sinceInstant, String endInstant) {
|
||||
Preconditions.checkNotNull(instantTime);
|
||||
Preconditions.checkArgument(HoodieTimeline.compareTimestamps(instantTime,
|
||||
sinceInstant, HoodieTimeline.GREATER),
|
||||
"Instant time(_hoodie_commit_time) in row (" + row + ") was : " + instantTime
|
||||
+ "but expected to be between " + sinceInstant + "(excl) - "
|
||||
+ endInstant + "(incl)");
|
||||
Preconditions.checkArgument(HoodieTimeline.compareTimestamps(instantTime,
|
||||
endInstant, HoodieTimeline.LESSER_OR_EQUAL),
|
||||
"Instant time(_hoodie_commit_time) in row (" + row + ") was : " + instantTime
|
||||
+ "but expected to be between " + sinceInstant + "(excl) - " + endInstant + "(incl)");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,266 @@
|
||||
/*
|
||||
* 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.utilities.sources.helpers;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.stream.Collectors;
|
||||
import kafka.common.TopicAndPartition;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.utilities.exception.HoodieDeltaStreamerException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.streaming.kafka.KafkaCluster;
|
||||
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset;
|
||||
import org.apache.spark.streaming.kafka.OffsetRange;
|
||||
import scala.Predef;
|
||||
import scala.collection.JavaConverters;
|
||||
import scala.collection.immutable.Map;
|
||||
import scala.collection.immutable.Set;
|
||||
import scala.collection.mutable.ArrayBuffer;
|
||||
import scala.collection.mutable.StringBuilder;
|
||||
import scala.util.Either;
|
||||
|
||||
|
||||
/**
|
||||
* Source to read data from Kafka, incrementally
|
||||
*/
|
||||
public class KafkaOffsetGen {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(KafkaOffsetGen.class);
|
||||
|
||||
private static long DEFAULT_MAX_EVENTS_TO_READ = 1000000; // 1M events max
|
||||
|
||||
public static class CheckpointUtils {
|
||||
|
||||
/**
|
||||
* Reconstruct checkpoint from string.
|
||||
*/
|
||||
public static HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> strToOffsets(
|
||||
String checkpointStr) {
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> offsetMap = new HashMap<>();
|
||||
if (checkpointStr.length() == 0) {
|
||||
return offsetMap;
|
||||
}
|
||||
String[] splits = checkpointStr.split(",");
|
||||
String topic = splits[0];
|
||||
for (int i = 1; i < splits.length; i++) {
|
||||
String[] subSplits = splits[i].split(":");
|
||||
offsetMap.put(new TopicAndPartition(topic, Integer.parseInt(subSplits[0])),
|
||||
new KafkaCluster.LeaderOffset("", -1, Long.parseLong(subSplits[1])));
|
||||
}
|
||||
return offsetMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* String representation of checkpoint <p> Format: topic1,0:offset0,1:offset1,2:offset2, .....
|
||||
*/
|
||||
public static String offsetsToStr(OffsetRange[] ranges) {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
// at least 1 partition will be present.
|
||||
sb.append(ranges[0].topic() + ",");
|
||||
sb.append(Arrays.stream(ranges)
|
||||
.map(r -> String.format("%s:%d", r.partition(), r.untilOffset()))
|
||||
.collect(Collectors.joining(",")));
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the offset ranges to read from Kafka, while handling newly added partitions, skews, event limits.
|
||||
*
|
||||
* @param fromOffsetMap offsets where we left off last time
|
||||
* @param toOffsetMap offsets of where each partitions is currently at
|
||||
* @param numEvents maximum number of events to read.
|
||||
*/
|
||||
public static OffsetRange[] computeOffsetRanges(
|
||||
HashMap<TopicAndPartition, LeaderOffset> fromOffsetMap,
|
||||
HashMap<TopicAndPartition, LeaderOffset> toOffsetMap,
|
||||
long numEvents) {
|
||||
|
||||
Comparator<OffsetRange> byPartition = Comparator.comparing(OffsetRange::partition);
|
||||
|
||||
// Create initial offset ranges for each 'to' partition, with from = to offsets.
|
||||
OffsetRange[] ranges = new OffsetRange[toOffsetMap.size()];
|
||||
toOffsetMap.entrySet().stream().map(e -> {
|
||||
TopicAndPartition tp = e.getKey();
|
||||
long fromOffset = fromOffsetMap.getOrDefault(tp, new LeaderOffset("", -1, 0)).offset();
|
||||
return OffsetRange.create(tp, fromOffset, fromOffset);
|
||||
}).sorted(byPartition).collect(Collectors.toList()).toArray(ranges);
|
||||
|
||||
long allocedEvents = 0;
|
||||
java.util.Set<Integer> exhaustedPartitions = new HashSet<>();
|
||||
// keep going until we have events to allocate and partitions still not exhausted.
|
||||
while (allocedEvents < numEvents && exhaustedPartitions.size() < toOffsetMap.size()) {
|
||||
long remainingEvents = numEvents - allocedEvents;
|
||||
long eventsPerPartition = (long) Math
|
||||
.ceil((1.0 * remainingEvents) / (toOffsetMap.size() - exhaustedPartitions.size()));
|
||||
|
||||
// Allocate the remaining events to non-exhausted partitions, in round robin fashion
|
||||
for (int i = 0; i < ranges.length; i++) {
|
||||
OffsetRange range = ranges[i];
|
||||
if (!exhaustedPartitions.contains(range.partition())) {
|
||||
long toOffsetMax = toOffsetMap.get(range.topicAndPartition()).offset();
|
||||
long toOffset = Math.min(toOffsetMax, range.untilOffset() + eventsPerPartition);
|
||||
if (toOffset == toOffsetMax) {
|
||||
exhaustedPartitions.add(range.partition());
|
||||
}
|
||||
allocedEvents += toOffset - range.untilOffset();
|
||||
ranges[i] = OffsetRange.create(range.topicAndPartition(), range.fromOffset(), toOffset);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return ranges;
|
||||
}
|
||||
|
||||
public static long totalNewMessages(OffsetRange[] ranges) {
|
||||
return Arrays.stream(ranges).mapToLong(OffsetRange::count).sum();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helpers to deal with tricky scala <=> java conversions. (oh my!)
|
||||
*/
|
||||
static class ScalaHelpers {
|
||||
|
||||
public static <K, V> Map<K, V> toScalaMap(HashMap<K, V> m) {
|
||||
return JavaConverters.mapAsScalaMapConverter(m).asScala().toMap(Predef.conforms());
|
||||
}
|
||||
|
||||
public static Set<String> toScalaSet(HashSet<String> s) {
|
||||
return JavaConverters.asScalaSetConverter(s).asScala().toSet();
|
||||
}
|
||||
|
||||
public static <K, V> java.util.Map<K, V> toJavaMap(Map<K, V> m) {
|
||||
return JavaConverters.mapAsJavaMapConverter(m).asJava();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Kafka reset offset strategies
|
||||
*/
|
||||
enum KafkaResetOffsetStrategies {
|
||||
LARGEST,
|
||||
SMALLEST
|
||||
}
|
||||
|
||||
/**
|
||||
* Configs to be passed for this source. All standard Kafka consumer configs are also respected
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private static final String KAFKA_TOPIC_NAME = "hoodie.deltastreamer.source.kafka.topic";
|
||||
private static final KafkaResetOffsetStrategies DEFAULT_AUTO_RESET_OFFSET = KafkaResetOffsetStrategies.LARGEST;
|
||||
}
|
||||
|
||||
private final HashMap<String, String> kafkaParams;
|
||||
private final TypedProperties props;
|
||||
protected final String topicName;
|
||||
|
||||
public KafkaOffsetGen(TypedProperties props) {
|
||||
this.props = props;
|
||||
kafkaParams = new HashMap<String, String>();
|
||||
for (Object prop : props.keySet()) {
|
||||
kafkaParams.put(prop.toString(), props.getString(prop.toString()));
|
||||
}
|
||||
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.KAFKA_TOPIC_NAME));
|
||||
topicName = props.getString(Config.KAFKA_TOPIC_NAME);
|
||||
}
|
||||
|
||||
public OffsetRange[] getNextOffsetRanges(Option<String> lastCheckpointStr, long sourceLimit) {
|
||||
|
||||
// Obtain current metadata for the topic
|
||||
KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams));
|
||||
Either<ArrayBuffer<Throwable>, Set<TopicAndPartition>> either = cluster.getPartitions(
|
||||
ScalaHelpers.toScalaSet(new HashSet<>(Collections.singletonList(topicName))));
|
||||
if (either.isLeft()) {
|
||||
// log errors. and bail out.
|
||||
throw new HoodieDeltaStreamerException("Error obtaining partition metadata",
|
||||
either.left().get().head());
|
||||
}
|
||||
Set<TopicAndPartition> topicPartitions = either.right().get();
|
||||
|
||||
// Determine the offset ranges to read from
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> fromOffsets;
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> checkpointOffsets;
|
||||
if (lastCheckpointStr.isPresent()) {
|
||||
fromOffsets = checkupValidOffsets(cluster, lastCheckpointStr, topicPartitions);
|
||||
} else {
|
||||
KafkaResetOffsetStrategies autoResetValue = KafkaResetOffsetStrategies.valueOf(
|
||||
props.getString("auto.offset.reset", Config.DEFAULT_AUTO_RESET_OFFSET.toString()).toUpperCase());
|
||||
switch (autoResetValue) {
|
||||
case SMALLEST:
|
||||
fromOffsets = new HashMap(ScalaHelpers.toJavaMap(
|
||||
cluster.getEarliestLeaderOffsets(topicPartitions).right().get()));
|
||||
break;
|
||||
case LARGEST:
|
||||
fromOffsets = new HashMap(
|
||||
ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||
break;
|
||||
default:
|
||||
throw new HoodieNotSupportedException(
|
||||
"Auto reset value must be one of 'smallest' or 'largest' ");
|
||||
}
|
||||
}
|
||||
|
||||
// Obtain the latest offsets.
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> toOffsets = new HashMap(
|
||||
ScalaHelpers.toJavaMap(cluster.getLatestLeaderOffsets(topicPartitions).right().get()));
|
||||
|
||||
// Come up with final set of OffsetRanges to read (account for new partitions, limit number of events)
|
||||
long numEvents = Math.min(DEFAULT_MAX_EVENTS_TO_READ, sourceLimit);
|
||||
OffsetRange[] offsetRanges = CheckpointUtils.computeOffsetRanges(fromOffsets, toOffsets, numEvents);
|
||||
|
||||
return offsetRanges;
|
||||
}
|
||||
|
||||
// check up checkpoint offsets is valid or not, if true, return checkpoint offsets,
|
||||
// else return earliest offsets
|
||||
private HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> checkupValidOffsets(
|
||||
KafkaCluster cluster,
|
||||
Option<String> lastCheckpointStr,
|
||||
Set<TopicAndPartition> topicPartitions) {
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> checkpointOffsets =
|
||||
CheckpointUtils.strToOffsets(lastCheckpointStr.get());
|
||||
HashMap<TopicAndPartition, KafkaCluster.LeaderOffset> earliestOffsets =
|
||||
new HashMap(ScalaHelpers.toJavaMap(
|
||||
cluster.getEarliestLeaderOffsets(topicPartitions).right().get()));
|
||||
|
||||
boolean checkpointOffsetReseter = checkpointOffsets.entrySet()
|
||||
.stream()
|
||||
.anyMatch(offset -> offset.getValue().offset()
|
||||
< earliestOffsets.get(offset.getKey()).offset());
|
||||
return checkpointOffsetReseter ? earliestOffsets : checkpointOffsets;
|
||||
}
|
||||
|
||||
|
||||
public String getTopicName() {
|
||||
return topicName;
|
||||
}
|
||||
|
||||
public HashMap<String, String> getKafkaParams() {
|
||||
return kafkaParams;
|
||||
}
|
||||
}
|
||||
@@ -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.utilities.transform;
|
||||
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
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;
|
||||
}
|
||||
}
|
||||
@@ -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.utilities.transform;
|
||||
|
||||
import java.util.UUID;
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
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;
|
||||
|
||||
/**
|
||||
* A transformer that allows a sql-query template be used to transform the source before writing to Hudi data-set.
|
||||
*
|
||||
* The query should reference the source as a table named "\<SRC\>"
|
||||
*/
|
||||
public class SqlQueryBasedTransformer implements Transformer {
|
||||
|
||||
private static volatile Logger log = LogManager.getLogger(SqlQueryBasedTransformer.class);
|
||||
|
||||
private static final String SRC_PATTERN = "<SRC>";
|
||||
private static final String TMP_TABLE = "HOODIE_SRC_TMP_TABLE_";
|
||||
|
||||
/**
|
||||
* Configs supported
|
||||
*/
|
||||
static class Config {
|
||||
|
||||
private static final String TRANSFORMER_SQL = "hoodie.deltastreamer.transformer.sql";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
|
||||
Dataset<Row> rowDataset, TypedProperties properties) {
|
||||
String transformerSQL = properties.getString(Config.TRANSFORMER_SQL);
|
||||
if (null == transformerSQL) {
|
||||
throw new IllegalArgumentException("Missing configuration : (" + Config.TRANSFORMER_SQL + ")");
|
||||
}
|
||||
|
||||
// tmp table name doesn't like dashes
|
||||
String tmpTable = TMP_TABLE.concat(UUID.randomUUID().toString().replace("-", "_"));
|
||||
log.info("Registering tmp table : " + tmpTable);
|
||||
rowDataset.registerTempTable(tmpTable);
|
||||
String sqlStr = transformerSQL.replaceAll(SRC_PATTERN, tmpTable);
|
||||
log.info("SQL Query for transformation : (" + sqlStr + ")");
|
||||
return sparkSession.sql(sqlStr);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* 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.utilities.transform;
|
||||
|
||||
import org.apache.hudi.common.util.TypedProperties;
|
||||
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;
|
||||
|
||||
/**
|
||||
* Transform source to target dataset before writing
|
||||
*/
|
||||
public interface Transformer {
|
||||
|
||||
/**
|
||||
* Transform source RDD to target RDD
|
||||
*
|
||||
* @param jsc JavaSparkContext
|
||||
* @param rowDataset Source DataSet
|
||||
* @param sparkSession Spark Session
|
||||
* @param properties Config properties
|
||||
* @return Transformed Dataset
|
||||
*/
|
||||
Dataset<Row> apply(JavaSparkContext jsc, SparkSession sparkSession,
|
||||
Dataset<Row> rowDataset, TypedProperties properties);
|
||||
}
|
||||
614
hudi-utilities/src/main/resources/META-INF/LICENSE.txt
Normal file
614
hudi-utilities/src/main/resources/META-INF/LICENSE.txt
Normal file
@@ -0,0 +1,614 @@
|
||||
|
||||
Apache License
|
||||
Version 2.0, January 2004
|
||||
http://www.apache.org/licenses/
|
||||
|
||||
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
|
||||
|
||||
1. Definitions.
|
||||
|
||||
"License" shall mean the terms and conditions for use, reproduction,
|
||||
and distribution as defined by Sections 1 through 9 of this document.
|
||||
|
||||
"Licensor" shall mean the copyright owner or entity authorized by
|
||||
the copyright owner that is granting the License.
|
||||
|
||||
"Legal Entity" shall mean the union of the acting entity and all
|
||||
other entities that control, are controlled by, or are under common
|
||||
control with that entity. For the purposes of this definition,
|
||||
"control" means (i) the power, direct or indirect, to cause the
|
||||
direction or management of such entity, whether by contract or
|
||||
otherwise, or (ii) ownership of fifty percent (50%) or more of the
|
||||
outstanding shares, or (iii) beneficial ownership of such entity.
|
||||
|
||||
"You" (or "Your") shall mean an individual or Legal Entity
|
||||
exercising permissions granted by this License.
|
||||
|
||||
"Source" form shall mean the preferred form for making modifications,
|
||||
including but not limited to software source code, documentation
|
||||
source, and configuration files.
|
||||
|
||||
"Object" form shall mean any form resulting from mechanical
|
||||
transformation or translation of a Source form, including but
|
||||
not limited to compiled object code, generated documentation,
|
||||
and conversions to other media types.
|
||||
|
||||
"Work" shall mean the work of authorship, whether in Source or
|
||||
Object form, made available under the License, as indicated by a
|
||||
copyright notice that is included in or attached to the work
|
||||
(an example is provided in the Appendix below).
|
||||
|
||||
"Derivative Works" shall mean any work, whether in Source or Object
|
||||
form, that is based on (or derived from) the Work and for which the
|
||||
editorial revisions, annotations, elaborations, or other modifications
|
||||
represent, as a whole, an original work of authorship. For the purposes
|
||||
of this License, Derivative Works shall not include works that remain
|
||||
separable from, or merely link (or bind by name) to the interfaces of,
|
||||
the Work and Derivative Works thereof.
|
||||
|
||||
"Contribution" shall mean any work of authorship, including
|
||||
the original version of the Work and any modifications or additions
|
||||
to that Work or Derivative Works thereof, that is intentionally
|
||||
submitted to Licensor for inclusion in the Work by the copyright owner
|
||||
or by an individual or Legal Entity authorized to submit on behalf of
|
||||
the copyright owner. For the purposes of this definition, "submitted"
|
||||
means any form of electronic, verbal, or written communication sent
|
||||
to the Licensor or its representatives, including but not limited to
|
||||
communication on electronic mailing lists, source code control systems,
|
||||
and issue tracking systems that are managed by, or on behalf of, the
|
||||
Licensor for the purpose of discussing and improving the Work, but
|
||||
excluding communication that is conspicuously marked or otherwise
|
||||
designated in writing by the copyright owner as "Not a Contribution."
|
||||
|
||||
"Contributor" shall mean Licensor and any individual or Legal Entity
|
||||
on behalf of whom a Contribution has been received by Licensor and
|
||||
subsequently incorporated within the Work.
|
||||
|
||||
2. Grant of Copyright License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
copyright license to reproduce, prepare Derivative Works of,
|
||||
publicly display, publicly perform, sublicense, and distribute the
|
||||
Work and such Derivative Works in Source or Object form.
|
||||
|
||||
3. Grant of Patent License. Subject to the terms and conditions of
|
||||
this License, each Contributor hereby grants to You a perpetual,
|
||||
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
|
||||
(except as stated in this section) patent license to make, have made,
|
||||
use, offer to sell, sell, import, and otherwise transfer the Work,
|
||||
where such license applies only to those patent claims licensable
|
||||
by such Contributor that are necessarily infringed by their
|
||||
Contribution(s) alone or by combination of their Contribution(s)
|
||||
with the Work to which such Contribution(s) was submitted. If You
|
||||
institute patent litigation against any entity (including a
|
||||
cross-claim or counterclaim in a lawsuit) alleging that the Work
|
||||
or a Contribution incorporated within the Work constitutes direct
|
||||
or contributory patent infringement, then any patent licenses
|
||||
granted to You under this License for that Work shall terminate
|
||||
as of the date such litigation is filed.
|
||||
|
||||
4. Redistribution. You may reproduce and distribute copies of the
|
||||
Work or Derivative Works thereof in any medium, with or without
|
||||
modifications, and in Source or Object form, provided that You
|
||||
meet the following conditions:
|
||||
|
||||
(a) You must give any other recipients of the Work or
|
||||
Derivative Works a copy of this License; and
|
||||
|
||||
(b) You must cause any modified files to carry prominent notices
|
||||
stating that You changed the files; and
|
||||
|
||||
(c) You must retain, in the Source form of any Derivative Works
|
||||
that You distribute, all copyright, patent, trademark, and
|
||||
attribution notices from the Source form of the Work,
|
||||
excluding those notices that do not pertain to any part of
|
||||
the Derivative Works; and
|
||||
|
||||
(d) If the Work includes a "NOTICE" text file as part of its
|
||||
distribution, then any Derivative Works that You distribute must
|
||||
include a readable copy of the attribution notices contained
|
||||
within such NOTICE file, excluding those notices that do not
|
||||
pertain to any part of the Derivative Works, in at least one
|
||||
of the following places: within a NOTICE text file distributed
|
||||
as part of the Derivative Works; within the Source form or
|
||||
documentation, if provided along with the Derivative Works; or,
|
||||
within a display generated by the Derivative Works, if and
|
||||
wherever such third-party notices normally appear. The contents
|
||||
of the NOTICE file are for informational purposes only and
|
||||
do not modify the License. You may add Your own attribution
|
||||
notices within Derivative Works that You distribute, alongside
|
||||
or as an addendum to the NOTICE text from the Work, provided
|
||||
that such additional attribution notices cannot be construed
|
||||
as modifying the License.
|
||||
|
||||
You may add Your own copyright statement to Your modifications and
|
||||
may provide additional or different license terms and conditions
|
||||
for use, reproduction, or distribution of Your modifications, or
|
||||
for any such Derivative Works as a whole, provided Your use,
|
||||
reproduction, and distribution of the Work otherwise complies with
|
||||
the conditions stated in this License.
|
||||
|
||||
5. Submission of Contributions. Unless You explicitly state otherwise,
|
||||
any Contribution intentionally submitted for inclusion in the Work
|
||||
by You to the Licensor shall be under the terms and conditions of
|
||||
this License, without any additional terms or conditions.
|
||||
Notwithstanding the above, nothing herein shall supersede or modify
|
||||
the terms of any separate license agreement you may have executed
|
||||
with Licensor regarding such Contributions.
|
||||
|
||||
6. Trademarks. This License does not grant permission to use the trade
|
||||
names, trademarks, service marks, or product names of the Licensor,
|
||||
except as required for reasonable and customary use in describing the
|
||||
origin of the Work and reproducing the content of the NOTICE file.
|
||||
|
||||
7. Disclaimer of Warranty. Unless required by applicable law or
|
||||
agreed to in writing, Licensor provides the Work (and each
|
||||
Contributor provides its Contributions) on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
|
||||
implied, including, without limitation, any warranties or conditions
|
||||
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
|
||||
PARTICULAR PURPOSE. You are solely responsible for determining the
|
||||
appropriateness of using or redistributing the Work and assume any
|
||||
risks associated with Your exercise of permissions under this License.
|
||||
|
||||
8. Limitation of Liability. In no event and under no legal theory,
|
||||
whether in tort (including negligence), contract, or otherwise,
|
||||
unless required by applicable law (such as deliberate and grossly
|
||||
negligent acts) or agreed to in writing, shall any Contributor be
|
||||
liable to You for damages, including any direct, indirect, special,
|
||||
incidental, or consequential damages of any character arising as a
|
||||
result of this License or out of the use or inability to use the
|
||||
Work (including but not limited to damages for loss of goodwill,
|
||||
work stoppage, computer failure or malfunction, or any and all
|
||||
other commercial damages or losses), even if such Contributor
|
||||
has been advised of the possibility of such damages.
|
||||
|
||||
9. Accepting Warranty or Additional Liability. While redistributing
|
||||
the Work or Derivative Works thereof, You may choose to offer,
|
||||
and charge a fee for, acceptance of support, warranty, indemnity,
|
||||
or other liability obligations and/or rights consistent with this
|
||||
License. However, in accepting such obligations, You may act only
|
||||
on Your own behalf and on Your sole responsibility, not on behalf
|
||||
of any other Contributor, and only if You agree to indemnify,
|
||||
defend, and hold each Contributor harmless for any liability
|
||||
incurred by, or claims asserted against, such Contributor by reason
|
||||
of your accepting any such warranty or additional liability.
|
||||
|
||||
END OF TERMS AND CONDITIONS
|
||||
|
||||
APPENDIX: How to apply the Apache License to your work.
|
||||
|
||||
To apply the Apache License to your work, attach the following
|
||||
boilerplate notice, with the fields enclosed by brackets "[]"
|
||||
replaced with your own identifying information. (Don't include
|
||||
the brackets!) The text should be enclosed in the appropriate
|
||||
comment syntax for the file format. We also recommend that a
|
||||
file or class name and description of purpose be included on the
|
||||
same "printed page" as the copyright notice for easier
|
||||
identification within third-party archives.
|
||||
|
||||
Copyright [yyyy] [name of copyright owner]
|
||||
|
||||
Licensed 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.
|
||||
|
||||
----
|
||||
This project bundles portions of the 'JQuery' project under the terms of the MIT license.
|
||||
|
||||
Copyright 2012 jQuery Foundation and other contributors
|
||||
http://jquery.com/
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining
|
||||
a copy of this software and associated documentation files (the
|
||||
"Software"), to deal in the Software without restriction, including
|
||||
without limitation the rights to use, copy, modify, merge, publish,
|
||||
distribute, sublicense, and/or sell copies of the Software, and to
|
||||
permit persons to whom the Software is furnished to do so, subject to
|
||||
the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be
|
||||
included in all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
|
||||
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
|
||||
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
|
||||
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
|
||||
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
|
||||
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
|
||||
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
|
||||
|
||||
----
|
||||
This project bundles a derivative of portions of the 'Asciidoctor' project
|
||||
under the terms of the MIT license.
|
||||
|
||||
The MIT License
|
||||
Copyright (C) 2012-2015 Dan Allen, Ryan Waldron and the Asciidoctor Project
|
||||
|
||||
Permission is hereby granted, free of charge, to any person obtaining a copy
|
||||
of this software and associated documentation files (the "Software"), to deal
|
||||
in the Software without restriction, including without limitation the rights
|
||||
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
|
||||
copies of the Software, and to permit persons to whom the Software is
|
||||
furnished to do so, subject to the following conditions:
|
||||
|
||||
The above copyright notice and this permission notice shall be included in
|
||||
all copies or substantial portions of the Software.
|
||||
|
||||
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
|
||||
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
|
||||
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
|
||||
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
|
||||
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
|
||||
THE SOFTWARE.
|
||||
|
||||
----
|
||||
This project incorporates portions of the 'Protocol Buffers' project avaialble
|
||||
under a '3-clause BSD' license.
|
||||
|
||||
Copyright 2008, Google Inc.
|
||||
All rights reserved.
|
||||
|
||||
Redistribution and use in source and binary forms, with or without
|
||||
modification, are permitted provided that the following conditions are
|
||||
met:
|
||||
|
||||
* Redistributions of source code must retain the above copyright
|
||||
notice, this list of conditions and the following disclaimer.
|
||||
* Redistributions in binary form must reproduce the above
|
||||
copyright notice, this list of conditions and the following disclaimer
|
||||
in the documentation and/or other materials provided with the
|
||||
distribution.
|
||||
* Neither the name of Google Inc. nor the names of its
|
||||
contributors may be used to endorse or promote products derived from
|
||||
this software without specific prior written permission.
|
||||
|
||||
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
|
||||
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
|
||||
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
|
||||
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
|
||||
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
|
||||
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
|
||||
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
|
||||
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
|
||||
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
|
||||
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
|
||||
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
|
||||
|
||||
Code generated by the Protocol Buffer compiler is owned by the owner
|
||||
of the input file used when generating it. This code is not
|
||||
standalone and requires a support library to be linked with it. This
|
||||
support library is itself covered by the above license.
|
||||
|
||||
----
|
||||
This project bundles a derivative image for our Orca Logo. This image is
|
||||
available under the Creative Commons By Attribution 3.0 License.
|
||||
|
||||
Creative Commons Legal Code
|
||||
|
||||
Attribution 3.0 Unported
|
||||
|
||||
CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE
|
||||
LEGAL SERVICES. DISTRIBUTION OF THIS LICENSE DOES NOT CREATE AN
|
||||
ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS
|
||||
INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES
|
||||
REGARDING THE INFORMATION PROVIDED, AND DISCLAIMS LIABILITY FOR
|
||||
DAMAGES RESULTING FROM ITS USE.
|
||||
|
||||
License
|
||||
|
||||
THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE
|
||||
COMMONS PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY
|
||||
COPYRIGHT AND/OR OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS
|
||||
AUTHORIZED UNDER THIS LICENSE OR COPYRIGHT LAW IS PROHIBITED.
|
||||
|
||||
BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE
|
||||
TO BE BOUND BY THE TERMS OF THIS LICENSE. TO THE EXTENT THIS LICENSE MAY
|
||||
BE CONSIDERED TO BE A CONTRACT, THE LICENSOR GRANTS YOU THE RIGHTS
|
||||
CONTAINED HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND
|
||||
CONDITIONS.
|
||||
|
||||
1. Definitions
|
||||
|
||||
a. "Adaptation" means a work based upon the Work, or upon the Work and
|
||||
other pre-existing works, such as a translation, adaptation,
|
||||
derivative work, arrangement of music or other alterations of a
|
||||
literary or artistic work, or phonogram or performance and includes
|
||||
cinematographic adaptations or any other form in which the Work may be
|
||||
recast, transformed, or adapted including in any form recognizably
|
||||
derived from the original, except that a work that constitutes a
|
||||
Collection will not be considered an Adaptation for the purpose of
|
||||
this License. For the avoidance of doubt, where the Work is a musical
|
||||
work, performance or phonogram, the synchronization of the Work in
|
||||
timed-relation with a moving image ("synching") will be considered an
|
||||
Adaptation for the purpose of this License.
|
||||
b. "Collection" means a collection of literary or artistic works, such as
|
||||
encyclopedias and anthologies, or performances, phonograms or
|
||||
broadcasts, or other works or subject matter other than works listed
|
||||
in Section 1(f) below, which, by reason of the selection and
|
||||
arrangement of their contents, constitute intellectual creations, in
|
||||
which the Work is included in its entirety in unmodified form along
|
||||
with one or more other contributions, each constituting separate and
|
||||
independent works in themselves, which together are assembled into a
|
||||
collective whole. A work that constitutes a Collection will not be
|
||||
considered an Adaptation (as defined above) for the purposes of this
|
||||
License.
|
||||
c. "Distribute" means to make available to the public the original and
|
||||
copies of the Work or Adaptation, as appropriate, through sale or
|
||||
other transfer of ownership.
|
||||
d. "Licensor" means the individual, individuals, entity or entities that
|
||||
offer(s) the Work under the terms of this License.
|
||||
e. "Original Author" means, in the case of a literary or artistic work,
|
||||
the individual, individuals, entity or entities who created the Work
|
||||
or if no individual or entity can be identified, the publisher; and in
|
||||
addition (i) in the case of a performance the actors, singers,
|
||||
musicians, dancers, and other persons who act, sing, deliver, declaim,
|
||||
play in, interpret or otherwise perform literary or artistic works or
|
||||
expressions of folklore; (ii) in the case of a phonogram the producer
|
||||
being the person or legal entity who first fixes the sounds of a
|
||||
performance or other sounds; and, (iii) in the case of broadcasts, the
|
||||
organization that transmits the broadcast.
|
||||
f. "Work" means the literary and/or artistic work offered under the terms
|
||||
of this License including without limitation any production in the
|
||||
literary, scientific and artistic domain, whatever may be the mode or
|
||||
form of its expression including digital form, such as a book,
|
||||
pamphlet and other writing; a lecture, address, sermon or other work
|
||||
of the same nature; a dramatic or dramatico-musical work; a
|
||||
choreographic work or entertainment in dumb show; a musical
|
||||
composition with or without words; a cinematographic work to which are
|
||||
assimilated works expressed by a process analogous to cinematography;
|
||||
a work of drawing, painting, architecture, sculpture, engraving or
|
||||
lithography; a photographic work to which are assimilated works
|
||||
expressed by a process analogous to photography; a work of applied
|
||||
art; an illustration, map, plan, sketch or three-dimensional work
|
||||
relative to geography, topography, architecture or science; a
|
||||
performance; a broadcast; a phonogram; a compilation of data to the
|
||||
extent it is protected as a copyrightable work; or a work performed by
|
||||
a variety or circus performer to the extent it is not otherwise
|
||||
considered a literary or artistic work.
|
||||
g. "You" means an individual or entity exercising rights under this
|
||||
License who has not previously violated the terms of this License with
|
||||
respect to the Work, or who has received express permission from the
|
||||
Licensor to exercise rights under this License despite a previous
|
||||
violation.
|
||||
h. "Publicly Perform" means to perform public recitations of the Work and
|
||||
to communicate to the public those public recitations, by any means or
|
||||
process, including by wire or wireless means or public digital
|
||||
performances; to make available to the public Works in such a way that
|
||||
members of the public may access these Works from a place and at a
|
||||
place individually chosen by them; to perform the Work to the public
|
||||
by any means or process and the communication to the public of the
|
||||
performances of the Work, including by public digital performance; to
|
||||
broadcast and rebroadcast the Work by any means including signs,
|
||||
sounds or images.
|
||||
i. "Reproduce" means to make copies of the Work by any means including
|
||||
without limitation by sound or visual recordings and the right of
|
||||
fixation and reproducing fixations of the Work, including storage of a
|
||||
protected performance or phonogram in digital form or other electronic
|
||||
medium.
|
||||
|
||||
2. Fair Dealing Rights. Nothing in this License is intended to reduce,
|
||||
limit, or restrict any uses free from copyright or rights arising from
|
||||
limitations or exceptions that are provided for in connection with the
|
||||
copyright protection under copyright law or other applicable laws.
|
||||
|
||||
3. License Grant. Subject to the terms and conditions of this License,
|
||||
Licensor hereby grants You a worldwide, royalty-free, non-exclusive,
|
||||
perpetual (for the duration of the applicable copyright) license to
|
||||
exercise the rights in the Work as stated below:
|
||||
|
||||
a. to Reproduce the Work, to incorporate the Work into one or more
|
||||
Collections, and to Reproduce the Work as incorporated in the
|
||||
Collections;
|
||||
b. to create and Reproduce Adaptations provided that any such Adaptation,
|
||||
including any translation in any medium, takes reasonable steps to
|
||||
clearly label, demarcate or otherwise identify that changes were made
|
||||
to the original Work. For example, a translation could be marked "The
|
||||
original work was translated from English to Spanish," or a
|
||||
modification could indicate "The original work has been modified.";
|
||||
c. to Distribute and Publicly Perform the Work including as incorporated
|
||||
in Collections; and,
|
||||
d. to Distribute and Publicly Perform Adaptations.
|
||||
e. For the avoidance of doubt:
|
||||
|
||||
i. Non-waivable Compulsory License Schemes. In those jurisdictions in
|
||||
which the right to collect royalties through any statutory or
|
||||
compulsory licensing scheme cannot be waived, the Licensor
|
||||
reserves the exclusive right to collect such royalties for any
|
||||
exercise by You of the rights granted under this License;
|
||||
ii. Waivable Compulsory License Schemes. In those jurisdictions in
|
||||
which the right to collect royalties through any statutory or
|
||||
compulsory licensing scheme can be waived, the Licensor waives the
|
||||
exclusive right to collect such royalties for any exercise by You
|
||||
of the rights granted under this License; and,
|
||||
iii. Voluntary License Schemes. The Licensor waives the right to
|
||||
collect royalties, whether individually or, in the event that the
|
||||
Licensor is a member of a collecting society that administers
|
||||
voluntary licensing schemes, via that society, from any exercise
|
||||
by You of the rights granted under this License.
|
||||
|
||||
The above rights may be exercised in all media and formats whether now
|
||||
known or hereafter devised. The above rights include the right to make
|
||||
such modifications as are technically necessary to exercise the rights in
|
||||
other media and formats. Subject to Section 8(f), all rights not expressly
|
||||
granted by Licensor are hereby reserved.
|
||||
|
||||
4. Restrictions. The license granted in Section 3 above is expressly made
|
||||
subject to and limited by the following restrictions:
|
||||
|
||||
a. You may Distribute or Publicly Perform the Work only under the terms
|
||||
of this License. You must include a copy of, or the Uniform Resource
|
||||
Identifier (URI) for, this License with every copy of the Work You
|
||||
Distribute or Publicly Perform. You may not offer or impose any terms
|
||||
on the Work that restrict the terms of this License or the ability of
|
||||
the recipient of the Work to exercise the rights granted to that
|
||||
recipient under the terms of the License. You may not sublicense the
|
||||
Work. You must keep intact all notices that refer to this License and
|
||||
to the disclaimer of warranties with every copy of the Work You
|
||||
Distribute or Publicly Perform. When You Distribute or Publicly
|
||||
Perform the Work, You may not impose any effective technological
|
||||
measures on the Work that restrict the ability of a recipient of the
|
||||
Work from You to exercise the rights granted to that recipient under
|
||||
the terms of the License. This Section 4(a) applies to the Work as
|
||||
incorporated in a Collection, but this does not require the Collection
|
||||
apart from the Work itself to be made subject to the terms of this
|
||||
License. If You create a Collection, upon notice from any Licensor You
|
||||
must, to the extent practicable, remove from the Collection any credit
|
||||
as required by Section 4(b), as requested. If You create an
|
||||
Adaptation, upon notice from any Licensor You must, to the extent
|
||||
practicable, remove from the Adaptation any credit as required by
|
||||
Section 4(b), as requested.
|
||||
b. If You Distribute, or Publicly Perform the Work or any Adaptations or
|
||||
Collections, You must, unless a request has been made pursuant to
|
||||
Section 4(a), keep intact all copyright notices for the Work and
|
||||
provide, reasonable to the medium or means You are utilizing: (i) the
|
||||
name of the Original Author (or pseudonym, if applicable) if supplied,
|
||||
and/or if the Original Author and/or Licensor designate another party
|
||||
or parties (e.g., a sponsor institute, publishing entity, journal) for
|
||||
attribution ("Attribution Parties") in Licensor's copyright notice,
|
||||
terms of service or by other reasonable means, the name of such party
|
||||
or parties; (ii) the title of the Work if supplied; (iii) to the
|
||||
extent reasonably practicable, the URI, if any, that Licensor
|
||||
specifies to be associated with the Work, unless such URI does not
|
||||
refer to the copyright notice or licensing information for the Work;
|
||||
and (iv) , consistent with Section 3(b), in the case of an Adaptation,
|
||||
a credit identifying the use of the Work in the Adaptation (e.g.,
|
||||
"French translation of the Work by Original Author," or "Screenplay
|
||||
based on original Work by Original Author"). The credit required by
|
||||
this Section 4 (b) may be implemented in any reasonable manner;
|
||||
provided, however, that in the case of a Adaptation or Collection, at
|
||||
a minimum such credit will appear, if a credit for all contributing
|
||||
authors of the Adaptation or Collection appears, then as part of these
|
||||
credits and in a manner at least as prominent as the credits for the
|
||||
other contributing authors. For the avoidance of doubt, You may only
|
||||
use the credit required by this Section for the purpose of attribution
|
||||
in the manner set out above and, by exercising Your rights under this
|
||||
License, You may not implicitly or explicitly assert or imply any
|
||||
connection with, sponsorship or endorsement by the Original Author,
|
||||
Licensor and/or Attribution Parties, as appropriate, of You or Your
|
||||
use of the Work, without the separate, express prior written
|
||||
permission of the Original Author, Licensor and/or Attribution
|
||||
Parties.
|
||||
c. Except as otherwise agreed in writing by the Licensor or as may be
|
||||
otherwise permitted by applicable law, if You Reproduce, Distribute or
|
||||
Publicly Perform the Work either by itself or as part of any
|
||||
Adaptations or Collections, You must not distort, mutilate, modify or
|
||||
take other derogatory action in relation to the Work which would be
|
||||
prejudicial to the Original Author's honor or reputation. Licensor
|
||||
agrees that in those jurisdictions (e.g. Japan), in which any exercise
|
||||
of the right granted in Section 3(b) of this License (the right to
|
||||
make Adaptations) would be deemed to be a distortion, mutilation,
|
||||
modification or other derogatory action prejudicial to the Original
|
||||
Author's honor and reputation, the Licensor will waive or not assert,
|
||||
as appropriate, this Section, to the fullest extent permitted by the
|
||||
applicable national law, to enable You to reasonably exercise Your
|
||||
right under Section 3(b) of this License (right to make Adaptations)
|
||||
but not otherwise.
|
||||
|
||||
5. Representations, Warranties and Disclaimer
|
||||
|
||||
UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR
|
||||
OFFERS THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY
|
||||
KIND CONCERNING THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE,
|
||||
INCLUDING, WITHOUT LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY,
|
||||
FITNESS FOR A PARTICULAR PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF
|
||||
LATENT OR OTHER DEFECTS, ACCURACY, OR THE PRESENCE OF ABSENCE OF ERRORS,
|
||||
WHETHER OR NOT DISCOVERABLE. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION
|
||||
OF IMPLIED WARRANTIES, SO SUCH EXCLUSION MAY NOT APPLY TO YOU.
|
||||
|
||||
6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE
|
||||
LAW, IN NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR
|
||||
ANY SPECIAL, INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES
|
||||
ARISING OUT OF THIS LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS
|
||||
BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
|
||||
|
||||
7. Termination
|
||||
|
||||
a. This License and the rights granted hereunder will terminate
|
||||
automatically upon any breach by You of the terms of this License.
|
||||
Individuals or entities who have received Adaptations or Collections
|
||||
from You under this License, however, will not have their licenses
|
||||
terminated provided such individuals or entities remain in full
|
||||
compliance with those licenses. Sections 1, 2, 5, 6, 7, and 8 will
|
||||
survive any termination of this License.
|
||||
b. Subject to the above terms and conditions, the license granted here is
|
||||
perpetual (for the duration of the applicable copyright in the Work).
|
||||
Notwithstanding the above, Licensor reserves the right to release the
|
||||
Work under different license terms or to stop distributing the Work at
|
||||
any time; provided, however that any such election will not serve to
|
||||
withdraw this License (or any other license that has been, or is
|
||||
required to be, granted under the terms of this License), and this
|
||||
License will continue in full force and effect unless terminated as
|
||||
stated above.
|
||||
|
||||
8. Miscellaneous
|
||||
|
||||
a. Each time You Distribute or Publicly Perform the Work or a Collection,
|
||||
the Licensor offers to the recipient a license to the Work on the same
|
||||
terms and conditions as the license granted to You under this License.
|
||||
b. Each time You Distribute or Publicly Perform an Adaptation, Licensor
|
||||
offers to the recipient a license to the original Work on the same
|
||||
terms and conditions as the license granted to You under this License.
|
||||
c. If any provision of this License is invalid or unenforceable under
|
||||
applicable law, it shall not affect the validity or enforceability of
|
||||
the remainder of the terms of this License, and without further action
|
||||
by the parties to this agreement, such provision shall be reformed to
|
||||
the minimum extent necessary to make such provision valid and
|
||||
enforceable.
|
||||
d. No term or provision of this License shall be deemed waived and no
|
||||
breach consented to unless such waiver or consent shall be in writing
|
||||
and signed by the party to be charged with such waiver or consent.
|
||||
e. This License constitutes the entire agreement between the parties with
|
||||
respect to the Work licensed here. There are no understandings,
|
||||
agreements or representations with respect to the Work not specified
|
||||
here. Licensor shall not be bound by any additional provisions that
|
||||
may appear in any communication from You. This License may not be
|
||||
modified without the mutual written agreement of the Licensor and You.
|
||||
f. The rights granted under, and the subject matter referenced, in this
|
||||
License were drafted utilizing the terminology of the Berne Convention
|
||||
for the Protection of Literary and Artistic Works (as amended on
|
||||
September 28, 1979), the Rome Convention of 1961, the WIPO Copyright
|
||||
Treaty of 1996, the WIPO Performances and Phonograms Treaty of 1996
|
||||
and the Universal Copyright Convention (as revised on July 24, 1971).
|
||||
These rights and subject matter take effect in the relevant
|
||||
jurisdiction in which the License terms are sought to be enforced
|
||||
according to the corresponding provisions of the implementation of
|
||||
those treaty provisions in the applicable national law. If the
|
||||
standard suite of rights granted under applicable copyright law
|
||||
includes additional rights not granted under this License, such
|
||||
additional rights are deemed to be included in the License; this
|
||||
License is not intended to restrict the license of any rights under
|
||||
applicable law.
|
||||
|
||||
|
||||
Creative Commons Notice
|
||||
|
||||
Creative Commons is not a party to this License, and makes no warranty
|
||||
whatsoever in connection with the Work. Creative Commons will not be
|
||||
liable to You or any party on any legal theory for any damages
|
||||
whatsoever, including without limitation any general, special,
|
||||
incidental or consequential damages arising in connection to this
|
||||
license. Notwithstanding the foregoing two (2) sentences, if Creative
|
||||
Commons has expressly identified itself as the Licensor hereunder, it
|
||||
shall have all rights and obligations of Licensor.
|
||||
|
||||
Except for the limited purpose of indicating to the public that the
|
||||
Work is licensed under the CCPL, Creative Commons does not authorize
|
||||
the use by either party of the trademark "Creative Commons" or any
|
||||
related trademark or logo of Creative Commons without the prior
|
||||
written consent of Creative Commons. Any permitted use will be in
|
||||
compliance with Creative Commons' then-current trademark usage
|
||||
guidelines, as may be published on its website or otherwise made
|
||||
available upon request from time to time. For the avoidance of doubt,
|
||||
this trademark restriction does not form part of this License.
|
||||
|
||||
Creative Commons may be contacted at https://creativecommons.org/.
|
||||
332
hudi-utilities/src/main/resources/META-INF/NOTICE.txt
Normal file
332
hudi-utilities/src/main/resources/META-INF/NOTICE.txt
Normal file
@@ -0,0 +1,332 @@
|
||||
Apache HUDI
|
||||
Copyright 2019 The Apache Software Foundation
|
||||
|
||||
Licensed 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.
|
||||
|
||||
This project includes:
|
||||
aircompressor under Apache License 2.0
|
||||
An open source Java toolkit for Amazon S3 under Apache License, Version 2.0
|
||||
Annotation 1.0 under The Apache Software License, Version 2.0
|
||||
ant under The Apache Software License, Version 2.0
|
||||
ANTLR 3 Runtime under BSD licence
|
||||
ANTLR 4 Runtime under The BSD License
|
||||
ANTLR ST4 4.0.4 under BSD licence
|
||||
ANTLR StringTemplate 4.0.2 under BSD licence
|
||||
AOP alliance under Public Domain
|
||||
aopalliance version 1.0 repackaged as a module under CDDL + GPLv2 with classpath exception
|
||||
Apache Ant Core under The Apache Software License, Version 2.0
|
||||
Apache Ant Launcher under The Apache Software License, Version 2.0
|
||||
Apache Avro under The Apache Software License, Version 2.0
|
||||
Apache Avro IPC under The Apache Software License, Version 2.0
|
||||
Apache Avro Mapred API under The Apache Software License, Version 2.0
|
||||
Apache Calcite Avatica under Apache License, Version 2.0
|
||||
Apache Calcite Avatica Metrics under Apache License, Version 2.0
|
||||
Apache Commons Collections under Apache License, Version 2.0
|
||||
Apache Commons Configuration under Apache License, Version 2.0
|
||||
Apache Commons Crypto under Apache License, Version 2.0
|
||||
Apache Commons IO under Apache License, Version 2.0
|
||||
Apache Commons Lang under Apache License, Version 2.0
|
||||
Apache Commons Logging under The Apache Software License, Version 2.0
|
||||
Apache Curator under The Apache Software License, Version 2.0
|
||||
Apache Derby Database Engine and Embedded JDBC Driver under Apache 2
|
||||
Apache Directory API ASN.1 API under The Apache Software License, Version 2.0
|
||||
Apache Directory LDAP API Utilities under The Apache Software License, Version 2.0
|
||||
Apache Groovy under The Apache Software License, Version 2.0
|
||||
Apache Hadoop Annotations under Apache License, Version 2.0
|
||||
Apache Hadoop Auth under Apache License, Version 2.0
|
||||
Apache Hadoop Client under Apache License, Version 2.0
|
||||
Apache Hadoop Common under Apache License, Version 2.0
|
||||
Apache Hadoop HDFS under Apache License, Version 2.0
|
||||
Apache HBase - Annotations under Apache License, Version 2.0
|
||||
Apache HBase - Client under Apache License, Version 2.0
|
||||
Apache HBase - Common under Apache License, Version 2.0
|
||||
Apache HBase - Protocol under Apache License, Version 2.0
|
||||
Apache HttpClient under Apache License, Version 2.0
|
||||
Apache HttpCore under Apache License, Version 2.0
|
||||
Apache Ivy under The Apache Software License, Version 2.0
|
||||
Apache Kafka under The Apache Software License, Version 2.0
|
||||
Apache Log4j under The Apache Software License, Version 2.0
|
||||
Apache Log4j 1.x Compatibility API under The Apache Software License, Version 2.0
|
||||
Apache Log4j API under The Apache Software License, Version 2.0
|
||||
Apache Log4j Core under The Apache Software License, Version 2.0
|
||||
Apache Log4j SLF4J Binding under The Apache Software License, Version 2.0
|
||||
Apache Log4j Web under The Apache Software License, Version 2.0
|
||||
Apache Parquet Avro under The Apache Software License, Version 2.0
|
||||
Apache Parquet Avro (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Column under The Apache Software License, Version 2.0
|
||||
Apache Parquet Column (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Common under The Apache Software License, Version 2.0
|
||||
Apache Parquet Common (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Encodings under The Apache Software License, Version 2.0
|
||||
Apache Parquet Encodings (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Format (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Generator (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop Bundle under The Apache Software License, Version 2.0
|
||||
Apache Parquet Hadoop Bundle (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Parquet Jackson under The Apache Software License, Version 2.0
|
||||
Apache Parquet Jackson (Incubating) under The Apache Software License, Version 2.0
|
||||
Apache Thrift under The Apache Software License, Version 2.0
|
||||
Apache Twill API under The Apache Software License, Version 2.0
|
||||
Apache Twill common library under The Apache Software License, Version 2.0
|
||||
Apache Twill core library under The Apache Software License, Version 2.0
|
||||
Apache Twill discovery service API under The Apache Software License, Version 2.0
|
||||
Apache Twill discovery service implementations under The Apache Software License, Version 2.0
|
||||
Apache Twill ZooKeeper client library under The Apache Software License, Version 2.0
|
||||
Apache Velocity under The Apache Software License, Version 2.0
|
||||
Apache XBean :: ASM 5 shaded (repackaged) under null or null
|
||||
ApacheDS I18n under The Apache Software License, Version 2.0
|
||||
ApacheDS Protocol Kerberos Codec under The Apache Software License, Version 2.0
|
||||
ASM Commons under 3-Clause BSD License
|
||||
ASM Core under 3-Clause BSD License
|
||||
ASM Tree under 3-Clause BSD License
|
||||
Bean Validation API under The Apache Software License, Version 2.0
|
||||
bijection-avro under Apache 2
|
||||
bijection-core under Apache 2
|
||||
BoneCP :: Core Library under Apache v2
|
||||
Calcite Core under Apache License, Version 2.0
|
||||
Calcite Druid under Apache License, Version 2.0
|
||||
Calcite Linq4j under Apache License, Version 2.0
|
||||
chill under Apache 2
|
||||
chill-java under Apache 2
|
||||
com.twitter.common:objectsize under Apache License, Version 2.0
|
||||
Commons BeanUtils Core under The Apache Software License, Version 2.0
|
||||
Commons CLI under The Apache Software License, Version 2.0
|
||||
Commons Codec under The Apache Software License, Version 2.0
|
||||
Commons Compiler under New BSD License
|
||||
Commons Compress under The Apache Software License, Version 2.0
|
||||
Commons Configuration under The Apache Software License, Version 2.0
|
||||
Commons Daemon under The Apache Software License, Version 2.0
|
||||
Commons DBCP under The Apache Software License, Version 2.0
|
||||
Commons Lang under The Apache Software License, Version 2.0
|
||||
Commons Math under The Apache Software License, Version 2.0
|
||||
Commons Net under The Apache Software License, Version 2.0
|
||||
Commons Pool under The Apache Software License, Version 2.0
|
||||
commons-beanutils under Apache License
|
||||
Compress-LZF under Apache License 2.0
|
||||
config under Apache License 2.0
|
||||
Curator Client under The Apache Software License, Version 2.0
|
||||
Curator Framework under The Apache Software License, Version 2.0
|
||||
Curator Recipes under The Apache Software License, Version 2.0
|
||||
Data Mapper for Jackson under The Apache Software License, Version 2.0
|
||||
DataNucleus Core under The Apache Software License, Version 2.0
|
||||
DataNucleus JDO API plugin under The Apache Software License, Version 2.0
|
||||
DataNucleus RDBMS plugin under The Apache Software License, Version 2.0
|
||||
Digester under The Apache Software License, Version 2.0
|
||||
Disruptor Framework under The Apache Software License, Version 2.0
|
||||
eigenbase-properties under Apache License, Version 2.0
|
||||
EL under The Apache Software License, Version 2.0
|
||||
empty under The Apache License, Version 2.0
|
||||
fastutil under Apache License, Version 2.0
|
||||
Findbugs Annotations under Apache License under Apache License, Version 2.0
|
||||
FindBugs-jsr305 under The Apache Software License, Version 2.0
|
||||
Fluent API for Apache HttpClient under Apache License, Version 2.0
|
||||
Glassfish Jasper under CDDL 1.0
|
||||
Glassfish Jasper API under Apache License Version 2.0
|
||||
Google Guice - Core Library under The Apache Software License, Version 2.0
|
||||
Google Guice - Extensions - AssistedInject under The Apache Software License, Version 2.0
|
||||
Google Guice - Extensions - Servlet under The Apache Software License, Version 2.0
|
||||
Graphite Integration for Metrics under Apache License 2.0
|
||||
Gson under The Apache Software License, Version 2.0
|
||||
Guava: Google Core Libraries for Java under The Apache Software License, Version 2.0
|
||||
Hadoop Metrics2 Reporter for Dropwizard Metrics under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-app under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-common under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-core under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-jobclient under Apache License, Version 2.0
|
||||
hadoop-mapreduce-client-shuffle under Apache License, Version 2.0
|
||||
hadoop-yarn-api under Apache License, Version 2.0
|
||||
hadoop-yarn-client under Apache License, Version 2.0
|
||||
hadoop-yarn-common under Apache License, Version 2.0
|
||||
hadoop-yarn-registry under Apache License, Version 2.0
|
||||
hadoop-yarn-server-applicationhistoryservice under Apache License, Version 2.0
|
||||
hadoop-yarn-server-common under Apache License, Version 2.0
|
||||
hadoop-yarn-server-resourcemanager under Apache License, Version 2.0
|
||||
hadoop-yarn-server-web-proxy under Apache License, Version 2.0
|
||||
Hamcrest Core under New BSD License
|
||||
HBase - Common under The Apache Software License, Version 2.0
|
||||
HBase - Hadoop Compatibility under The Apache Software License, Version 2.0
|
||||
HBase - Hadoop Two Compatibility under The Apache Software License, Version 2.0
|
||||
HBase - Prefix Tree under The Apache Software License, Version 2.0
|
||||
HBase - Procedure under The Apache Software License, Version 2.0
|
||||
HBase - Server under The Apache Software License, Version 2.0
|
||||
HikariCP under The Apache Software License, Version 2.0
|
||||
Hive Common under The Apache Software License, Version 2.0
|
||||
Hive JDBC under The Apache Software License, Version 2.0
|
||||
Hive Llap Client under The Apache Software License, Version 2.0
|
||||
Hive Llap Common under The Apache Software License, Version 2.0
|
||||
Hive Llap Server under The Apache Software License, Version 2.0
|
||||
Hive Llap Tez under The Apache Software License, Version 2.0
|
||||
Hive Metastore under The Apache Software License, Version 2.0
|
||||
Hive Query Language under The Apache Software License, Version 2.0
|
||||
Hive Serde under The Apache Software License, Version 2.0
|
||||
Hive Service under The Apache Software License, Version 2.0
|
||||
Hive Service RPC under The Apache Software License, Version 2.0
|
||||
Hive Shims under The Apache Software License, Version 2.0
|
||||
Hive Shims 0.23 under The Apache Software License, Version 2.0
|
||||
Hive Shims Common under The Apache Software License, Version 2.0
|
||||
Hive Shims Scheduler under The Apache Software License, Version 2.0
|
||||
Hive Storage API under Apache License, Version 2.0
|
||||
Hive Vector-Code-Gen Utilities under The Apache Software License, Version 2.0
|
||||
HK2 API module under CDDL + GPLv2 with classpath exception
|
||||
HK2 Implementation Utilities under CDDL + GPLv2 with classpath exception
|
||||
hoodie-client under Apache License, Version 2.0
|
||||
hoodie-common under Apache License, Version 2.0
|
||||
hoodie-hadoop-mr under Apache License, Version 2.0
|
||||
hoodie-hive under Apache License, Version 2.0
|
||||
hoodie-spark under Apache License, Version 2.0
|
||||
hoodie-timeline-service under Apache License, Version 2.0
|
||||
hoodie-utilities under Apache License, Version 2.0
|
||||
htrace-core under The Apache Software License, Version 2.0
|
||||
HttpClient under Apache License
|
||||
IntelliJ IDEA Annotations under The Apache Software License, Version 2.0
|
||||
Jackson under The Apache Software License, Version 2.0
|
||||
Jackson Integration for Metrics under Apache License 2.0
|
||||
Jackson-annotations under The Apache Software License, Version 2.0
|
||||
Jackson-core under The Apache Software License, Version 2.0
|
||||
jackson-databind under The Apache Software License, Version 2.0
|
||||
Jackson-module-paranamer under The Apache Software License, Version 2.0
|
||||
jackson-module-scala under The Apache Software License, Version 2.0
|
||||
jamon-runtime under Mozilla Public License Version 1.1
|
||||
Janino under New BSD License
|
||||
jasper-compiler under The Apache Software License, Version 2.0
|
||||
jasper-runtime under The Apache Software License, Version 2.0
|
||||
Java Authentication SPI for Containers under The Apache Software License, Version 2.0
|
||||
Java Servlet API under CDDL + GPLv2 with classpath exception
|
||||
java-xmlbuilder under Apache License, Version 2.0
|
||||
JavaBeans Activation Framework (JAF) under Common Development and Distribution License (CDDL) v1.0
|
||||
Javalin under The Apache Software License, Version 2.0
|
||||
JavaMail API under Common Development and Distribution License (CDDL) v1.0
|
||||
Javassist under MPL 1.1 or LGPL 2.1 or Apache License 2.0
|
||||
javax.annotation API under CDDL + GPLv2 with classpath exception
|
||||
javax.inject under The Apache Software License, Version 2.0
|
||||
javax.inject:1 as OSGi bundle under CDDL + GPLv2 with classpath exception
|
||||
javax.ws.rs-api under CDDL 1.1 or GPL2 w/ CPE
|
||||
Javolution under BSD License
|
||||
JAX-RS provider for JSON content type under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
|
||||
JAXB RI under CDDL 1.1 or GPL2 w/ CPE
|
||||
JCL 1.1.1 implemented over SLF4J under MIT License
|
||||
JCodings under MIT License
|
||||
jcommander under Apache 2.0
|
||||
JDO API under Apache 2
|
||||
jersey-client under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-container-servlet under CDDL+GPL License
|
||||
jersey-container-servlet-core under CDDL+GPL License
|
||||
jersey-core under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-core-client under CDDL+GPL License
|
||||
jersey-core-common under CDDL+GPL License
|
||||
jersey-core-server under CDDL+GPL License
|
||||
jersey-guice under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-json under CDDL 1.1 or GPL2 w/ CPE
|
||||
jersey-media-jaxb under CDDL+GPL License
|
||||
jersey-repackaged-guava under CDDL+GPL License
|
||||
jersey-server under CDDL 1.1 or GPL2 w/ CPE
|
||||
Jettison under Apache License, Version 2.0
|
||||
Jetty :: Aggregate :: All core Jetty under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Asynchronous HTTP Client under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Continuation under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Http Utility under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: IO Utility under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Security under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Server Core under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Servlet Handling under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Webapp Application Support under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Client under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Common under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: Websocket :: Servlet Interface under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty :: XML utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty Orbit :: Servlet API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
Jetty SSLEngine under Apache License Version 2
|
||||
Jetty Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
|
||||
JLine under The BSD License
|
||||
Joda-Time under Apache 2
|
||||
Joni under MIT License
|
||||
JPam under The Apache Software License, Version 2.0
|
||||
JSch under BSD
|
||||
json4s-ast under ASL
|
||||
json4s-core under ASL
|
||||
json4s-jackson under ASL
|
||||
jsp-api under CDDL
|
||||
JTA 1.1 under The Apache Software License, Version 2.0
|
||||
JUL to SLF4J bridge under MIT License
|
||||
JUnit under Common Public License Version 1.0
|
||||
JVM Integration for Metrics under Apache License 2.0
|
||||
kafka-avro-serializer under Apache License 2.0
|
||||
kafka-schema-registry-client under Apache License 2.0
|
||||
Kryo Shaded under 3-Clause BSD License
|
||||
leveldbjni-all under The BSD 3-Clause License
|
||||
LZ4 and xxHash under The Apache Software License, Version 2.0
|
||||
Metrics Core under Apache License 2.0
|
||||
Metrics Core Library under Apache License 2.0
|
||||
MinLog under New BSD License
|
||||
Mockito under The MIT License
|
||||
Netty/All-in-One under Apache License, Version 2.0
|
||||
Objenesis under Apache 2
|
||||
Open JSON under The Apache Software License, Version 2.0
|
||||
opencsv under Apache 2
|
||||
ORC Core under Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-common under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-jdk7 under The Apache License, Version 2.0
|
||||
org.jetbrains.kotlin:kotlin-stdlib-jdk8 under The Apache License, Version 2.0
|
||||
org.pentaho:pentaho-aggdesigner-algorithm under Apache License, Version 2.0
|
||||
oro under Apache License, Version 2.0
|
||||
OSGi resource locator bundle - used by various API providers that rely on META-INF/services mechanism to locate providers. under CDDL + GPLv2 with classpath exception
|
||||
ParaNamer Core under BSD
|
||||
Protocol Buffer Java API under New BSD license
|
||||
Py4J under The New BSD License
|
||||
pyrolite under MIT License
|
||||
RabbitMQ Java Client under ASL 2.0 or GPL v2 or MPL 1.1
|
||||
RoaringBitmap under Apache 2
|
||||
RocksDB JNI under Apache License 2.0 or GNU General Public License, version 2
|
||||
Scala Compiler under BSD 3-Clause
|
||||
Scala Library under BSD 3-Clause
|
||||
scala-parser-combinators under BSD 3-clause
|
||||
scala-xml under BSD 3-clause
|
||||
Scalap under BSD 3-Clause
|
||||
scalatest under the Apache License, ASL Version 2.0
|
||||
ServiceLocator Default Implementation under CDDL + GPLv2 with classpath exception
|
||||
Servlet Specification 2.5 API under CDDL 1.0
|
||||
Servlet Specification API under Apache License Version 2.0
|
||||
servlet-api under CDDL
|
||||
SLF4J API Module under MIT License
|
||||
SLF4J LOG4J-12 Binding under MIT License
|
||||
Slider Core under Apache License, Version 2.0
|
||||
snappy-java under The Apache Software License, Version 2.0
|
||||
Spark Integration for Kafka 0.8 under Apache 2.0 License
|
||||
Spark Project Catalyst under Apache 2.0 License
|
||||
Spark Project Core under Apache 2.0 License
|
||||
Spark Project Launcher under Apache 2.0 License
|
||||
Spark Project Networking under Apache 2.0 License
|
||||
Spark Project Shuffle Streaming Service under Apache 2.0 License
|
||||
Spark Project Sketch under Apache 2.0 License
|
||||
Spark Project SQL under Apache 2.0 License
|
||||
Spark Project Streaming under Apache 2.0 License
|
||||
Spark Project Tags under Apache 2.0 License
|
||||
Spark Project Unsafe under Apache 2.0 License
|
||||
spark-avro under Apache-2.0
|
||||
StAX API under The Apache Software License, Version 2.0
|
||||
stream-lib under Apache License, Version 2.0
|
||||
Tephra API under The Apache Software License, Version 2.0
|
||||
Tephra Core under The Apache Software License, Version 2.0
|
||||
Tephra HBase 1.0 Compatibility under The Apache Software License, Version 2.0
|
||||
The Netty Project under Apache License, Version 2.0
|
||||
univocity-parsers under Apache 2
|
||||
utils under Apache License 2.0
|
||||
Xerces2 Java Parser under The Apache Software License, Version 2.0
|
||||
XML Commons External Components XML APIs under The Apache Software License, Version 2.0
|
||||
Xml Compatibility extensions for Jackson under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
|
||||
xmlenc Library under The BSD License
|
||||
XZ for Java under Public Domain
|
||||
ZkClient under The Apache Software License, Version 2.0
|
||||
zookeeper under Apache License, Version 2.0
|
||||
|
||||
Reference in New Issue
Block a user