1
0

[HUDI-845] Added locking capability to allow multiple writers (#2374)

* [HUDI-845] Added locking capability to allow multiple writers
1. Added LockProvider API for pluggable lock methodologies
2. Added Resolution Strategy API to allow for pluggable conflict resolution
3. Added TableService client API to schedule table services
4. Added Transaction Manager for wrapping actions within transactions
This commit is contained in:
n3nash
2021-03-16 16:43:53 -07:00
committed by GitHub
parent b038623ed3
commit 74241947c1
88 changed files with 4876 additions and 381 deletions

View File

@@ -23,13 +23,13 @@ import org.apache.hudi.DataSourceUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.DagUtils;
import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator;
@@ -38,6 +38,8 @@ import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode;
import org.apache.hudi.integ.testsuite.dag.scheduler.DagScheduler;
import org.apache.hudi.integ.testsuite.dag.scheduler.SaferSchemaDagScheduler;
import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
import org.apache.hudi.integ.testsuite.helpers.ZookeeperServiceProvider;
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
import org.apache.hudi.keygen.BuiltinKeyGenerator;
@@ -182,9 +184,9 @@ public class HoodieTestSuiteJob {
long startTime = System.currentTimeMillis();
WriterContext writerContext = new WriterContext(jsc, props, cfg, keyGenerator, sparkSession);
writerContext.initContext(jsc);
startOtherServicesIfNeeded(writerContext);
if (this.cfg.saferSchemaEvolution) {
int numRollbacks = 2; // rollback most recent upsert/insert, by default.
// if root is RollbackNode, get num_rollbacks
List<DagNode> root = workflowDag.getNodeList();
if (!root.isEmpty() && root.get(0) instanceof RollbackNode) {
@@ -203,8 +205,31 @@ public class HoodieTestSuiteJob {
log.error("Failed to run Test Suite ", e);
throw new HoodieException("Failed to run Test Suite ", e);
} finally {
stopQuietly();
}
}
private void stopQuietly() {
try {
sparkSession.stop();
jsc.stop();
} catch (Exception e) {
log.error("Unable to stop spark session", e);
}
}
private void startOtherServicesIfNeeded(WriterContext writerContext) throws Exception {
if (cfg.startHiveMetastore) {
HiveServiceProvider hiveServiceProvider = new HiveServiceProvider(
Config.newBuilder().withHiveLocal(true).build());
hiveServiceProvider.startLocalHiveServiceIfNeeded(writerContext.getHoodieTestSuiteWriter().getConfiguration());
hiveServiceProvider.syncToLocalHiveIfNeeded(writerContext.getHoodieTestSuiteWriter());
}
if (cfg.startZookeeper) {
ZookeeperServiceProvider zookeeperServiceProvider = new ZookeeperServiceProvider(Config.newBuilder().withHiveLocal(true).build(),
writerContext.getHoodieTestSuiteWriter().getConfiguration());
zookeeperServiceProvider.startLocalZookeeperIfNeeded();
}
}
@@ -262,5 +287,11 @@ public class HoodieTestSuiteJob {
+ "(If not provided, assumed to be false.)",
required = false)
public Boolean saferSchemaEvolution = false;
@Parameter(names = {"--start-zookeeper"}, description = "Start Zookeeper instance to use for optimistic lock ")
public Boolean startZookeeper = false;
@Parameter(names = {"--start-hive-metastore"}, description = "Start Hive Metastore to use for optimistic lock ")
public Boolean startHiveMetastore = false;
}
}

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.integ.testsuite;
import java.io.Serializable;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.SparkRDDWriteClient;
@@ -28,7 +31,6 @@ import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieClusteringConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodiePayloadConfig;
@@ -40,12 +42,12 @@ import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.RollbackNode;
import org.apache.hudi.integ.testsuite.dag.nodes.ScheduleCompactNode;
import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.utilities.schema.SchemaProvider;
import org.apache.hadoop.conf.Configuration;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.rdd.RDD;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -60,7 +62,7 @@ import java.util.Set;
* A writer abstraction for the Hudi test suite. This class wraps different implementations of writers used to perform write operations into the target hudi dataset. Current supported writers are
* {@link HoodieDeltaStreamerWrapper} and {@link SparkRDDWriteClient}.
*/
public class HoodieTestSuiteWriter {
public class HoodieTestSuiteWriter implements Serializable {
private static Logger log = LoggerFactory.getLogger(HoodieTestSuiteWriter.class);
@@ -121,6 +123,14 @@ public class HoodieTestSuiteWriter {
return false;
}
public RDD<GenericRecord> getNextBatch() throws Exception {
Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> nextBatch = fetchSource();
lastCheckpoint = Option.of(nextBatch.getValue().getLeft());
JavaRDD <HoodieRecord> inputRDD = nextBatch.getRight().getRight();
return inputRDD.map(r -> (GenericRecord) r.getData()
.getInsertValue(new Schema.Parser().parse(schema)).get()).rdd();
}
public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchSource() throws Exception {
return this.deltaStreamerWrapper.fetchSource();
}
@@ -253,4 +263,12 @@ public class HoodieTestSuiteWriter {
public Option<String> getLastCheckpoint() {
return lastCheckpoint;
}
public Properties getProps() {
return props;
}
public String getSchema() {
return schema;
}
}

View File

@@ -109,4 +109,8 @@ public class WriterContext {
public String toString() {
return this.hoodieTestSuiteWriter.toString() + "\n" + this.deltaGenerator.toString() + "\n";
}
public SparkSession getSparkSession() {
return sparkSession;
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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.integ.testsuite.helpers;
import org.apache.curator.framework.CuratorFramework;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
public class ZookeeperServiceProvider {
Configuration configuration;
private CuratorFramework client;
private Config config;
public ZookeeperServiceProvider(Config config, Configuration configuration) {
this.configuration = configuration;
this.config = config;
}
public void startLocalZookeeperIfNeeded() throws Exception {
if (config.isHiveLocal()) {
ZookeeperTestService zookeeperTestService = new ZookeeperTestService(configuration);
zookeeperTestService.start();
}
}
public void stopLocalZookeeperIfNeeded() throws Exception {
if (config.isHiveLocal()) {
ZookeeperTestService zookeeperTestService = new ZookeeperTestService(configuration);
zookeeperTestService.stop();
}
}
}

View File

@@ -49,7 +49,6 @@ public class DFSDeltaWriterAdapter implements DeltaWriterAdapter<GenericRecord>
public List<DeltaWriteStats> write(Iterator<GenericRecord> input) throws IOException {
while (input.hasNext()) {
GenericRecord next = input.next();
next.put(SchemaUtils.SOURCE_ORDERING_FIELD, preCombineFieldVal);
if (this.deltaInputWriter.canWrite()) {
this.deltaInputWriter.writeData(next);
} else {

View File

@@ -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.integ.testsuite.dag.nodes
import org.apache.hudi.client.WriteStatus
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config
import org.apache.hudi.integ.testsuite.dag.ExecutionContext
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SaveMode
import scala.collection.JavaConverters._
/**
* Spark datasource based insert node
* @param config1
*/
class SparkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
config = config1
/**
* Execute the {@link DagNode}.
*
* @param context The context needed for an execution of a node.
* @param curItrCount iteration count for executing the node.
* @throws Exception Thrown if the execution failed.
*/
override def execute(context: ExecutionContext, curItrCount: Int): Unit = {
if (!config.isDisableGenerate) {
println("Generating input data for node {}", this.getName)
context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count()
}
val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch,
context.getWriterContext.getHoodieTestSuiteWriter.getSchema,
context.getWriterContext.getSparkSession)
inputDF.write.format("hudi")
.options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap))
.option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(SaveMode.Overwrite)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}
}

View File

@@ -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.integ.testsuite.dag.nodes
import org.apache.hudi.client.WriteStatus
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config
import org.apache.hudi.integ.testsuite.dag.ExecutionContext
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SaveMode
import scala.collection.JavaConverters._
/**
* Spark datasource based upsert node
* @param config1
*/
class SparkUpsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
config = config1
/**
* Execute the {@link DagNode}.
*
* @param context The context needed for an execution of a node.
* @param curItrCount iteration count for executing the node.
* @throws Exception Thrown if the execution failed.
*/
override def execute(context: ExecutionContext, curItrCount: Int): Unit = {
if (!config.isDisableGenerate) {
println("Generating input data for node {}", this.getName)
context.getDeltaGenerator().writeRecords(context.getDeltaGenerator().generateInserts(config)).count()
}
val inputDF = AvroConversionUtils.createDataFrame(context.getWriterContext.getHoodieTestSuiteWriter.getNextBatch,
context.getWriterContext.getHoodieTestSuiteWriter.getSchema,
context.getWriterContext.getSparkSession)
inputDF.write.format("hudi")
.options(DataSourceWriteOptions.translateSqlOptions(context.getWriterContext.getProps.asScala.toMap))
.option(DataSourceWriteOptions.TABLE_NAME_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, context.getHoodieTestSuiteWriter.getCfg.tableType)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX_OPT_KEY, "deltastreamer.checkpoint.key")
.option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
.option(HoodieWriteConfig.TABLE_NAME, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
.mode(SaveMode.Append)
.save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
}
}