1
0

Test Suite should work with Docker + Unit Tests

This commit is contained in:
Abhishek Modi
2020-09-01 20:12:22 -07:00
committed by n3nash
parent 2fee087f0f
commit 53d1e55110
21 changed files with 422 additions and 185 deletions

View File

@@ -60,15 +60,14 @@ first_hive_sync:
first_hive_query:
config:
hive_props:
prop1: "set hive.execution.engine=spark"
prop2: "set spark.yarn.queue="
prop3: "set hive.strict.checks.large.query=false"
prop4: "set hive.stats.autogather=false"
hive_queries:
query1: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1"
query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1"
result1: 0
query2: "select count(*) from testdb1.table1"
result2: 22100000
query2: "select count(*) from testdb.table1"
result2: 11600
type: HiveQueryNode
deps: first_hive_sync
second_upsert:
@@ -84,14 +83,13 @@ second_upsert:
second_hive_query:
config:
hive_props:
prop1: "set hive.execution.engine=mr"
prop2: "set mapred.job.queue.name="
prop3: "set hive.strict.checks.large.query=false"
prop4: "set hive.stats.autogather=false"
hive_queries:
query1: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1"
query1: "select count(*) from testdb.table1 group by `_row_key` having count(*) > 1"
result1: 0
query2: "select count(*) from testdb1.table1"
result2: 22100
query2: "select count(*) from testdb.table1"
result2: 11900
type: HiveQueryNode
deps: second_upsert
deps: second_upsert

View File

@@ -13,15 +13,25 @@
# 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.
# write configs
hoodie.datasource.write.recordkey.field=_row_key
hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-bench/input
hoodie.datasource.write.keygenerator.class=org.apache.hudi.ComplexKeyGenerator
hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator
hoodie.datasource.write.partitionpath.field=timestamp
hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc
# deltastreamer configs
hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd
hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP
hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-bench/input
hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/test-suite/source.avsc
hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc
#hive sync
hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/
hoodie.datasource.hive_sync.database=testdb
hoodie.datasource.hive_sync.table=test_table
hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor
hoodie.datasource.hive_sync.table=table1
hoodie.datasource.hive_sync.use_jdbc=false
hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
hoodie.datasource.hive_sync.assume_date_partitioning=true
hoodie.datasource.write.keytranslator.class=org.apache.hudi.DayBasedPartitionPathKeyTranslator
hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/bench/source.avsc
hoodie.datasource.hive_sync.use_pre_apache_input_format=true

View File

@@ -101,6 +101,25 @@ public class FSUtils {
return fs;
}
public static FileSystem getFs(String path, Configuration conf, boolean localByDefault) {
if (localByDefault) {
return getFs(addSchemeIfLocalPath(path).toString(), conf);
}
return getFs(path, conf);
}
public static Path addSchemeIfLocalPath(String path) {
Path providedPath = new Path(path);
File localFile = new File(path);
if (!providedPath.isAbsolute() && localFile.exists()) {
Path resolvedPath = new Path("file://" + localFile.getAbsolutePath());
LOG.info("Resolving file " + path + " to be a local file.");
return resolvedPath;
}
LOG.info("Resolving file " + path + "to be a remote file.");
return providedPath;
}
/**
* A write token uniquely identifies an attempt at one of the IOHandle operations (Merge/Create/Append).
*/

View File

@@ -51,13 +51,13 @@ The test suite supports different types of operations besides ingestion such as
## Entry class to the test suite
```
org.apache.hudi.testsuite.HoodieTestSuiteJob.java - Entry Point of the hudi test suite job. This
org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.java - Entry Point of the hudi test suite job. This
class wraps all the functionalities required to run a configurable integration suite.
```
## Configurations required to run the job
```
org.apache.hudi.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig - Config class that drives the behavior of the
org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig - Config class that drives the behavior of the
integration test suite. This class extends from com.uber.hoodie.utilities.DeltaStreamerConfig. Look at
link#HudiDeltaStreamer page to learn about all the available configs applicable to your test suite.
```
@@ -68,29 +68,29 @@ There are 2 ways to generate a workload pattern
1.Programmatically
Choose to write up the entire DAG of operations programmatically, take a look at `WorkflowDagGenerator` class.
You can create a DAG of operations programmatically - take a look at `WorkflowDagGenerator` class.
Once you're ready with the DAG you want to execute, simply pass the class name as follows:
```
spark-submit
...
...
--class org.apache.hudi.testsuite.HoodieTestSuiteJob
--workload-generator-classname org.apache.hudi.testsuite.dag.scheduler.<your_workflowdaggenerator>
--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob
--workload-generator-classname org.apache.hudi.integ.testsuite.dag.scheduler.<your_workflowdaggenerator>
...
```
2.YAML file
Choose to write up the entire DAG of operations in YAML, take a look at `complex-workload-dag-cow.yaml` or
`complex-workload-dag-mor.yaml`.
Choose to write up the entire DAG of operations in YAML, take a look at `complex-dag-cow.yaml` or
`complex-dag-mor.yaml`.
Once you're ready with the DAG you want to execute, simply pass the yaml file path as follows:
```
spark-submit
...
...
--class org.apache.hudi.testsuite.HoodieTestSuiteJob
--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob
--workload-yaml-path /path/to/your-workflow-dag.yaml
...
```
@@ -136,127 +136,161 @@ Take a look at the `TestHoodieTestSuiteJob` to check how you can run the entire
## Running an end to end test suite in Local Docker environment
Start the Hudi Docker demo:
```
docker/setup_demo.sh
```
NOTE: We need to make a couple of environment changes for Hive 2.x support. This will be fixed once Hudi moves to Spark 3.x
```
docker exec -it adhoc-2 bash
cd /opt/spark/jars
rm /opt/spark/jars/hive*
rm spark-hive-thriftserver_2.11-2.4.4.jar
wget https://repo1.maven.org/maven2/org/apache/spark/spark-hive-thriftserver_2.12/3.0.0-preview2/spark-hive-thriftserver_2.12-3.0.0-preview2.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-common/2.3.1/hive-common-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-exec/2.3.1/hive-exec-2.3.1-core.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-jdbc/2.3.1/hive-jdbc-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-llap-common/2.3.1/hive-llap-common-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-metastore/2.3.1/hive-metastore-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-serde/2.3.1/hive-serde-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-service/2.3.1/hive-service-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-service-rpc/2.3.1/hive-service-rpc-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/shims/hive-shims-0.23/2.3.1/hive-shims-0.23-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/shims/hive-shims-common/2.3.1/hive-shims-common-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-storage-api/2.3.1/hive-storage-api-2.3.1.jar
wget https://repo1.maven.org/maven2/org/apache/hive/hive-shims/2.3.1/hive-shims-2.3.1.jar
wget https://repo1.maven.org/maven2/org/json/json/20090211/json-20090211.jar
cp /opt/hive/lib/log* /opt/spark/jars/
rm log4j-slf4j-impl-2.6.2.jar
cd /opt
```
Copy the integration tests jar into the docker container
```
docker cp packaging/hudi-integ-test-bundle/target/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar adhoc-2:/opt
```
Copy the following test properties file:
```
echo '
hoodie.deltastreamer.source.test.num_partitions=100
hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false
hoodie.deltastreamer.source.test.max_unique_records=100000000
hoodie.embed.timeline.server=false
hoodie.datasource.write.recordkey.field=_row_key
hoodie.datasource.write.keygenerator.class=org.apache.hudi.keygen.TimestampBasedKeyGenerator
hoodie.datasource.write.partitionpath.field=timestamp
hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-integ-test-suite/input
hoodie.deltastreamer.schemaprovider.target.schema.file=file:/var/hoodie/ws/docker/demo/config/test-suite/source.avsc
hoodie.deltastreamer.schemaprovider.source.schema.file=file:/var/hoodie/ws/docker/demo/config/test-suite/source.avsc
hoodie.deltastreamer.keygen.timebased.timestamp.type=UNIX_TIMESTAMP
hoodie.deltastreamer.keygen.timebased.output.dateformat=yyyy/MM/dd
hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/
hoodie.datasource.hive_sync.database=testdb
hoodie.datasource.hive_sync.table=table1
hoodie.datasource.hive_sync.assume_date_partitioning=false
hoodie.datasource.hive_sync.partition_fields=_hoodie_partition_path
hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
' > test.properties
docker cp test.properties adhoc-2:/opt
```
Clean the working directories before starting a new test:
```
hdfs dfs -rm -r /user/hive/warehouse/hudi-integ-test-suite/output/
hdfs dfs -rm -r /user/hive/warehouse/hudi-integ-test-suite/input/
```
Launch a Copy-on-Write job:
```
docker exec -it adhoc-2 /bin/bash
# COPY_ON_WRITE tables
=========================
## Run the following command to start the test suite
spark-submit \
--packages com.databricks:spark-avro_2.11:4.0.0 \
--conf spark.task.cpus=1 \
--conf spark.executor.cores=1 \
--conf spark.task.maxFailures=100 \
--conf spark.memory.fraction=0.4 \
--conf spark.rdd.compress=true \
--conf spark.kryoserializer.buffer.max=2000m \
--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \
--conf spark.memory.storageFraction=0.1 \
--conf spark.shuffle.service.enabled=true \
--conf spark.sql.hive.convertMetastoreParquet=false \
--conf spark.ui.port=5555 \
--conf spark.driver.maxResultSize=12g \
--conf spark.executor.heartbeatInterval=120s \
--conf spark.network.timeout=600s \
--conf spark.eventLog.overwrite=true \
--conf spark.eventLog.enabled=true \
--conf spark.yarn.max.executor.failures=10 \
--conf spark.sql.catalogImplementation=hive \
--conf spark.sql.shuffle.partitions=1000 \
--class org.apache.hudi.testsuite.HoodieTestSuiteJob $HUDI_TEST_SUITE_BUNDLE \
--source-ordering-field timestamp \
--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \
--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \
--target-table test_table \
--props /var/hoodie/ws/docker/demo/config/testsuite/test-source.properties \
--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \
--source-limit 300000 \
--source-class org.apache.hudi.utilities.sources.AvroDFSSource \
--input-file-size 125829120 \
--workload-yaml-path /var/hoodie/ws/docker/demo/config/testsuite/complex-workflow-dag-cow.yaml \
--storage-type COPY_ON_WRITE \
--compact-scheduling-minshare 1 \
--hoodie-conf "hoodie.deltastreamer.source.test.num_partitions=100" \
--hoodie-conf "hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false" \
--hoodie-conf "hoodie.deltastreamer.source.test.max_unique_records=100000000" \
--hoodie-conf "hoodie.embed.timeline.server=false" \
--hoodie-conf "hoodie.datasource.write.recordkey.field=_row_key" \
--hoodie-conf "hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-integ-test-suite/input" \
--hoodie-conf "hoodie.datasource.write.keygenerator.class=org.apache.hudi.ComplexKeyGenerator" \
--hoodie-conf "hoodie.datasource.write.partitionpath.field=timestamp" \
--hoodie-conf "hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc" \
--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=false" \
--hoodie-conf "hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/" \
--hoodie-conf "hoodie.datasource.hive_sync.database=testdb" \
--hoodie-conf "hoodie.datasource.hive_sync.table=test_table" \
--hoodie-conf "hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor" \
--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=true" \
--hoodie-conf "hoodie.datasource.write.keytranslator.class=org.apache.hudi.DayBasedPartitionPathKeyTranslator" \
--hoodie-conf "hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc"
...
...
2019-11-03 05:44:47 INFO DagScheduler:69 - ----------- Finished workloads ----------
2019-11-03 05:44:47 INFO HoodieTestSuiteJob:138 - Finished scheduling all tasks
...
2019-11-03 05:44:48 INFO SparkContext:54 - Successfully stopped SparkContext
spark-submit \
--packages org.apache.spark:spark-avro_2.11:2.4.0 \
--conf spark.task.cpus=1 \
--conf spark.executor.cores=1 \
--conf spark.task.maxFailures=100 \
--conf spark.memory.fraction=0.4 \
--conf spark.rdd.compress=true \
--conf spark.kryoserializer.buffer.max=2000m \
--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \
--conf spark.memory.storageFraction=0.1 \
--conf spark.shuffle.service.enabled=true \
--conf spark.sql.hive.convertMetastoreParquet=false \
--conf spark.driver.maxResultSize=12g \
--conf spark.executor.heartbeatInterval=120s \
--conf spark.network.timeout=600s \
--conf spark.yarn.max.executor.failures=10 \
--conf spark.sql.catalogImplementation=hive \
--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \
/opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \
--source-ordering-field timestamp \
--use-deltastreamer \
--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \
--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \
--target-table table1 \
--props test.properties \
--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \
--source-class org.apache.hudi.utilities.sources.AvroDFSSource \
--input-file-size 125829120 \
--workload-yaml-path file:/var/hoodie/ws/docker/demo/config/test-suite/complex-dag-cow.yaml \
--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \
--table-type COPY_ON_WRITE \
--compact-scheduling-minshare 1
```
Or a Merge-on-Read job:
```
# MERGE_ON_READ tables
=========================
## Run the following command to start the test suite
spark-submit \
--packages com.databricks:spark-avro_2.11:4.0.0 \
--conf spark.task.cpus=1 \
--conf spark.executor.cores=1 \
--conf spark.task.maxFailures=100 \
--conf spark.memory.fraction=0.4 \
--conf spark.rdd.compress=true \
--conf spark.kryoserializer.buffer.max=2000m \
--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \
--conf spark.memory.storageFraction=0.1 \
--conf spark.shuffle.service.enabled=true \
--conf spark.sql.hive.convertMetastoreParquet=false \
--conf spark.ui.port=5555 \
--conf spark.driver.maxResultSize=12g \
--conf spark.executor.heartbeatInterval=120s \
--conf spark.network.timeout=600s \
--conf spark.eventLog.overwrite=true \
--conf spark.eventLog.enabled=true \
--conf spark.yarn.max.executor.failures=10 \
--conf spark.sql.catalogImplementation=hive \
--conf spark.sql.shuffle.partitions=1000 \
--class org.apache.hudi.testsuite.HoodieTestSuiteJob $HUDI_TEST_SUITE_BUNDLE \
--source-ordering-field timestamp \
--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \
--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \
--target-table test_table \
--props /var/hoodie/ws/docker/demo/config/testsuite/test-source.properties \
--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \
--source-limit 300000 \
--source-class org.apache.hudi.utilities.sources.AvroDFSSource \
--input-file-size 125829120 \
--workload-yaml-path /var/hoodie/ws/docker/demo/config/testsuite/complex-workflow-dag-mor.yaml \
--storage-type MERGE_ON_READ \
--compact-scheduling-minshare 1 \
--hoodie-conf "hoodie.deltastreamer.source.test.num_partitions=100" \
--hoodie-conf "hoodie.deltastreamer.source.test.datagen.use_rocksdb_for_storing_existing_keys=false" \
--hoodie-conf "hoodie.deltastreamer.source.test.max_unique_records=100000000" \
--hoodie-conf "hoodie.embed.timeline.server=false" \
--hoodie-conf "hoodie.datasource.write.recordkey.field=_row_key" \
--hoodie-conf "hoodie.deltastreamer.source.dfs.root=/user/hive/warehouse/hudi-integ-test-suite/input" \
--hoodie-conf "hoodie.datasource.write.keygenerator.class=org.apache.hudi.ComplexKeyGenerator" \
--hoodie-conf "hoodie.datasource.write.partitionpath.field=timestamp" \
--hoodie-conf "hoodie.deltastreamer.schemaprovider.source.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc" \
--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=false" \
--hoodie-conf "hoodie.datasource.hive_sync.jdbcurl=jdbc:hive2://hiveserver:10000/" \
--hoodie-conf "hoodie.datasource.hive_sync.database=testdb" \
--hoodie-conf "hoodie.datasource.hive_sync.table=test_table" \
--hoodie-conf "hoodie.datasource.hive_sync.partition_extractor_class=org.apache.hudi.hive.NonPartitionedExtractor" \
--hoodie-conf "hoodie.datasource.hive_sync.assume_date_partitioning=true" \
--hoodie-conf "hoodie.datasource.write.keytranslator.class=org.apache.hudi.DayBasedPartitionPathKeyTranslator" \
--hoodie-conf "hoodie.deltastreamer.schemaprovider.target.schema.file=/var/hoodie/ws/docker/demo/config/testsuite/source.avsc"
...
...
2019-11-03 05:44:47 INFO DagScheduler:69 - ----------- Finished workloads ----------
2019-11-03 05:44:47 INFO HoodieTestSuiteJob:138 - Finished scheduling all tasks
...
2019-11-03 05:44:48 INFO SparkContext:54 - Successfully stopped SparkContext
spark-submit \
--packages org.apache.spark:spark-avro_2.11:2.4.0 \
--conf spark.task.cpus=1 \
--conf spark.executor.cores=1 \
--conf spark.task.maxFailures=100 \
--conf spark.memory.fraction=0.4 \
--conf spark.rdd.compress=true \
--conf spark.kryoserializer.buffer.max=2000m \
--conf spark.serializer=org.apache.spark.serializer.KryoSerializer \
--conf spark.memory.storageFraction=0.1 \
--conf spark.shuffle.service.enabled=true \
--conf spark.sql.hive.convertMetastoreParquet=false \
--conf spark.driver.maxResultSize=12g \
--conf spark.executor.heartbeatInterval=120s \
--conf spark.network.timeout=600s \
--conf spark.yarn.max.executor.failures=10 \
--conf spark.sql.catalogImplementation=hive \
--class org.apache.hudi.integ.testsuite.HoodieTestSuiteJob \
/opt/hudi-integ-test-bundle-0.6.1-SNAPSHOT.jar \
--source-ordering-field timestamp \
--use-deltastreamer \
--target-base-path /user/hive/warehouse/hudi-integ-test-suite/output \
--input-base-path /user/hive/warehouse/hudi-integ-test-suite/input \
--target-table table1 \
--props test.properties \
--schemaprovider-class org.apache.hudi.utilities.schema.FilebasedSchemaProvider \
--source-class org.apache.hudi.utilities.sources.AvroDFSSource \
--input-file-size 125829120 \
--workload-yaml-path file:/var/hoodie/ws/docker/demo/config/test-suite/complex-dag-mor.yaml \
--workload-generator-classname org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator \
--table-type MERGE_ON_READ \
--compact-scheduling-minshare 1
```

View File

@@ -90,6 +90,7 @@ public class HoodieTestSuiteJob {
public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc) throws IOException {
this.cfg = cfg;
this.jsc = jsc;
cfg.propsFilePath = FSUtils.addSchemeIfLocalPath(cfg.propsFilePath).toString();
this.sparkSession = SparkSession.builder().config(jsc.getConf()).getOrCreate();
this.fs = FSUtils.getFs(cfg.inputBasePath, jsc.hadoopConfiguration());
this.props = UtilHelpers.readConfig(fs, new Path(cfg.propsFilePath), cfg.configs).getConfig();
@@ -123,11 +124,18 @@ public class HoodieTestSuiteJob {
new HoodieTestSuiteJob(cfg, jssc).runTestSuite();
}
public WorkflowDag createWorkflowDag() throws IOException {
WorkflowDag workflowDag = this.cfg.workloadYamlPath == null ? ((WorkflowDagGenerator) ReflectionUtils
.loadClass((this.cfg).workloadDagGenerator)).build()
: DagUtils.convertYamlPathToDag(
FSUtils.getFs(this.cfg.workloadYamlPath, jsc.hadoopConfiguration(), true),
this.cfg.workloadYamlPath);
return workflowDag;
}
public void runTestSuite() {
try {
WorkflowDag workflowDag = this.cfg.workloadYamlPath == null ? ((WorkflowDagGenerator) ReflectionUtils
.loadClass((this.cfg).workloadDagGenerator)).build()
: DagUtils.convertYamlPathToDag(this.fs, this.cfg.workloadYamlPath);
WorkflowDag workflowDag = createWorkflowDag();
log.info("Workflow Dag => " + DagUtils.convertDagToYaml(workflowDag));
long startTime = System.currentTimeMillis();
String schemaStr = schemaProvider.getSourceSchema().toString();
@@ -143,6 +151,7 @@ public class HoodieTestSuiteJob {
log.error("Failed to run Test Suite ", e);
throw new HoodieException("Failed to run Test Suite ", e);
} finally {
sparkSession.stop();
jsc.stop();
}
}

View File

@@ -39,7 +39,6 @@ public class HiveSyncNode extends DagNode<Boolean> {
log.info("Executing hive sync node");
this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive();
this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
}

View File

@@ -80,7 +80,7 @@ public class DagScheduler {
private void execute(ExecutorService service, List<DagNode> nodes) throws Exception {
// Nodes at the same level are executed in parallel
Queue<DagNode> queue = new PriorityQueue<>(nodes);
log.info("Running workloads");
log.warn("Running workloads");
do {
List<Future> futures = new ArrayList<>();
Set<DagNode> childNodes = new HashSet<>();
@@ -110,6 +110,7 @@ public class DagScheduler {
throw new RuntimeException("DagNode already completed! Cannot re-execute");
}
try {
log.warn("executing node: " + node.getName() + " of type: " + node.getClass());
node.execute(executionContext);
node.setCompleted(true);
log.info("Finished executing {}", node.getName());

View File

@@ -94,11 +94,12 @@ public class DeltaGenerator implements Serializable {
public JavaRDD<GenericRecord> generateInserts(Config operation) {
long recordsPerPartition = operation.getNumRecordsInsert();
int numPartitions = operation.getNumInsertPartitions();
int minPayloadSize = operation.getRecordSize();
JavaRDD<GenericRecord> inputBatch = jsc.parallelize(Collections.EMPTY_LIST)
.repartition(operation.getNumInsertPartitions()).mapPartitions(p -> {
return new LazyRecordGeneratorIterator(new FlexibleSchemaRecordGenerationIterator(recordsPerPartition,
minPayloadSize, schemaStr, partitionPathFieldNames));
minPayloadSize, schemaStr, partitionPathFieldNames, numPartitions));
});
return inputBatch;
}

View File

@@ -40,15 +40,15 @@ public class FlexibleSchemaRecordGenerationIterator implements Iterator<GenericR
private List<String> partitionPathFieldNames;
public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, String schema) {
this(maxEntriesToProduce, GenericRecordFullPayloadGenerator.DEFAULT_PAYLOAD_SIZE, schema, null);
this(maxEntriesToProduce, GenericRecordFullPayloadGenerator.DEFAULT_PAYLOAD_SIZE, schema, null, GenericRecordFullPayloadGenerator.DEFAULT_NUM_DATE_PARTITIONS);
}
public FlexibleSchemaRecordGenerationIterator(long maxEntriesToProduce, int minPayloadSize, String schemaStr,
List<String> partitionPathFieldNames) {
List<String> partitionPathFieldNames, int numPartitions) {
this.counter = maxEntriesToProduce;
this.partitionPathFieldNames = partitionPathFieldNames;
Schema schema = new Schema.Parser().parse(schemaStr);
this.generator = new GenericRecordFullPayloadGenerator(schema, minPayloadSize);
this.generator = new GenericRecordFullPayloadGenerator(schema, minPayloadSize, numPartitions);
}
@Override

View File

@@ -27,6 +27,8 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Type;
import org.apache.avro.generic.GenericData;
@@ -44,6 +46,7 @@ import org.slf4j.LoggerFactory;
public class GenericRecordFullPayloadGenerator implements Serializable {
public static final int DEFAULT_PAYLOAD_SIZE = 1024 * 10; // 10 KB
public static final int DEFAULT_NUM_DATE_PARTITIONS = 50;
private static Logger log = LoggerFactory.getLogger(GenericRecordFullPayloadGenerator.class);
protected final Random random = new Random();
// The source schema used to generate a payload
@@ -58,6 +61,8 @@ public class GenericRecordFullPayloadGenerator implements Serializable {
private int numberOfComplexFields;
// The size of a full record where every field of a generic record created contains 1 random value
private int estimatedFullPayloadSize;
// The number of unique dates to create
private int numDatePartitions = DEFAULT_NUM_DATE_PARTITIONS;
// LogicalTypes in Avro 1.8.2
private static final String DECIMAL = "decimal";
private static final String UUID_NAME = "uuid";
@@ -86,6 +91,10 @@ public class GenericRecordFullPayloadGenerator implements Serializable {
}
}
}
public GenericRecordFullPayloadGenerator(Schema schema, int minPayloadSize, int numDatePartitions) {
this(schema, minPayloadSize);
this.numDatePartitions = numDatePartitions;
}
protected static boolean isPrimitive(Schema localSchema) {
if (localSchema.getType() != Type.ARRAY
@@ -170,6 +179,12 @@ public class GenericRecordFullPayloadGenerator implements Serializable {
return record;
}
private long getNextConstrainedLong() {
int numPartitions = random.nextInt(numDatePartitions);
long unixTimeStamp = TimeUnit.SECONDS.convert(numPartitions, TimeUnit.DAYS);
return unixTimeStamp;
}
/**
* Generate random value according to their type.
*/
@@ -188,7 +203,7 @@ public class GenericRecordFullPayloadGenerator implements Serializable {
case INT:
return random.nextInt();
case LONG:
return random.nextLong();
return getNextConstrainedLong();
case STRING:
return UUID.randomUUID().toString();
case ENUM:

View File

@@ -41,11 +41,11 @@ public class HiveSyncDagGenerator implements WorkflowDagGenerator {
.withNumTimesToRepeat(1)
.withRecordSize(1000).build());
DagNode child1 = new HiveSyncNode(Config.newBuilder().withHiveLocal(true).build());
DagNode child1 = new HiveSyncNode(Config.newBuilder().build());
root.addChildNode(child1);
DagNode child2 = new HiveQueryNode(Config.newBuilder().withHiveLocal(true).withHiveQueryAndResults(Arrays
DagNode child2 = new HiveQueryNode(Config.newBuilder().withHiveQueryAndResults(Arrays
.asList(Pair.of("select " + "count(*) from testdb1.table1 group " + "by rider having count(*) < 1", 0)))
.build());
child1.addChildNode(child2);

View File

@@ -34,7 +34,7 @@ import org.junit.jupiter.api.Test;
*/
public class TestDagUtils {
private static final String COW_DAG_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/complex-dag-cow.yaml";
private static final String COW_DAG_DOCKER_DEMO_RELATIVE_PATH = "/hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml";
@Test
public void testConvertDagToYaml() throws Exception {

View File

@@ -42,6 +42,7 @@ import org.apache.hudi.utilities.sources.AvroDFSSource;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
@@ -59,8 +60,13 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
+ ".properties";
private static final String SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/source.avsc";
private static final String TARGET_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/target.avsc";
private static final String COW_DAG_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/complex-dag-cow.yaml";
private static final String MOR_DAG_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/complex-dag-mor.yaml";
private static final String COW_DAG_FILE_NAME = "unit-test-cow-dag.yaml";
private static final String COW_DAG_SOURCE_PATH = "/hudi-integ-test/src/test/resources/" + COW_DAG_FILE_NAME;
private static final String MOR_DAG_FILE_NAME = "unit-test-mor-dag.yaml";
private static final String MOR_DAG_SOURCE_PATH = "/hudi-integ-test/src/test/resources/" + MOR_DAG_FILE_NAME;
public static Stream<Arguments> configParams() {
Object[][] data =
@@ -80,9 +86,9 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
+ TARGET_SCHEMA_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/target.avsc");
UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+ COW_DAG_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/complex-dag-cow.yaml");
+ COW_DAG_SOURCE_PATH, dfs, dfsBasePath + "/" + COW_DAG_FILE_NAME);
UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+ MOR_DAG_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/complex-dag-mor.yaml");
+ MOR_DAG_SOURCE_PATH, dfs, dfsBasePath + "/" + MOR_DAG_FILE_NAME);
TypedProperties props = new TypedProperties();
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
@@ -93,6 +99,7 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc");
props.setProperty("hoodie.deltastreamer.source.dfs.root", dfsBasePath + "/input");
props.setProperty("hoodie.datasource.hive_sync.assume_date_partitioning", "true");
props.setProperty("hoodie.datasource.hive_sync.skip_ro_suffix", "true");
props.setProperty("hoodie.datasource.write.keytranslator.class", "org.apache.hudi"
+ ".DayBasedPartitionPathKeyTranslator");
props.setProperty("hoodie.compact.inline.max.delta.commits", "3");
@@ -138,13 +145,17 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
super.teardown();
}
private void cleanDFSDirs() throws Exception {
dfs.delete(new Path(dfsBasePath + "/input"), true);
dfs.delete(new Path(dfsBasePath + "/result"), true);
}
// Tests in this class add to the test build time significantly. Since this is a Integration Test (end to end), we
// would like to run this as a nightly build which is a TODO.
// TODO : Clean up input / result paths after each test
@MethodSource("configParams")
public void testDagWithInsertUpsertAndValidate(boolean useDeltaStreamer, String tableType) throws Exception {
dfs.delete(new Path(dfsBasePath + "/input"), true);
dfs.delete(new Path(dfsBasePath + "/result"), true);
this.cleanDFSDirs();
String inputBasePath = dfsBasePath + "/input/" + UUID.randomUUID().toString();
String outputBasePath = dfsBasePath + "/result/" + UUID.randomUUID().toString();
HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, tableType);
@@ -155,10 +166,11 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 2);
}
@MethodSource("configParams")
public void testHiveSync(boolean useDeltaStreamer, String tableType) throws Exception {
dfs.delete(new Path(dfsBasePath + "/input"), true);
dfs.delete(new Path(dfsBasePath + "/result"), true);
@Test
public void testHiveSync() throws Exception {
boolean useDeltaStreamer = false;
String tableType = "COPY_ON_WRITE";
this.cleanDFSDirs();
String inputBasePath = dfsBasePath + "/input";
String outputBasePath = dfsBasePath + "/result";
HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, tableType);
@@ -173,34 +185,34 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 1);
}
@MethodSource("configParams")
public void testCOWFullDagFromYaml(boolean useDeltaStreamer, String tableType) throws Exception {
dfs.delete(new Path(dfsBasePath + "/input"), true);
dfs.delete(new Path(dfsBasePath + "/result"), true);
@Test
public void testCOWFullDagFromYaml() throws Exception {
boolean useDeltaStreamer = false;
this.cleanDFSDirs();
String inputBasePath = dfsBasePath + "/input";
String outputBasePath = dfsBasePath + "/result";
HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, HoodieTableType
.COPY_ON_WRITE.name());
cfg.workloadYamlPath = dfsBasePath + "/complex-dag-cow.yaml";
cfg.workloadYamlPath = dfsBasePath + "/" + COW_DAG_FILE_NAME;
HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
hoodieTestSuiteJob.runTestSuite();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 5);
//assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 5);
}
@MethodSource("configParams")
public void testMORFullDagFromYaml(boolean useDeltaStreamer, String tableType) throws Exception {
dfs.delete(new Path(dfsBasePath + "/input"), true);
dfs.delete(new Path(dfsBasePath + "/result"), true);
@Test
public void testMORFullDagFromYaml() throws Exception {
boolean useDeltaStreamer = false;
this.cleanDFSDirs();
String inputBasePath = dfsBasePath + "/input";
String outputBasePath = dfsBasePath + "/result";
HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, HoodieTableType
.MERGE_ON_READ.name());
cfg.workloadYamlPath = dfsBasePath + "/complex-dag-mor.yaml";
cfg.workloadYamlPath = dfsBasePath + "/" + MOR_DAG_FILE_NAME;
HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
hoodieTestSuiteJob.runTestSuite();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7);
//assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7);
}
protected HoodieTestSuiteConfig makeConfig(String inputBasePath, String outputBasePath, boolean useDeltaStream,

View File

@@ -0,0 +1,40 @@
# 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.
first_insert:
config:
record_size: 70000
num_insert_partitions: 1
repeat_count: 1
num_records_insert: 100
type: InsertNode
deps: none
first_hive_sync:
config:
queue_name: "adhoc"
engine: "mr"
type: HiveSyncNode
deps: first_insert
first_hive_query:
config:
hive_props:
prop2: "set spark.yarn.queue="
prop3: "set hive.strict.checks.large.query=false"
prop4: "set hive.stats.autogather=false"
hive_queries:
query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1"
result2: 0
type: HiveQueryNode
deps: first_hive_sync

View File

@@ -0,0 +1,40 @@
# 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.
first_insert:
config:
record_size: 70000
num_insert_partitions: 1
repeat_count: 1
num_records_insert: 100
type: InsertNode
deps: none
first_hive_sync:
config:
queue_name: "adhoc"
engine: "mr"
type: HiveSyncNode
deps: first_insert
first_hive_query:
config:
hive_props:
prop2: "set spark.yarn.queue="
prop3: "set hive.strict.checks.large.query=false"
prop4: "set hive.stats.autogather=false"
hive_queries:
query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1"
result2: 0
type: HiveQueryNode
deps: first_hive_sync

View File

@@ -254,6 +254,8 @@ public class DataSourceUtils {
SlashEncodedDayPartitionValueExtractor.class.getName());
hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL()));
hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX(),
DataSourceWriteOptions.DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL()));
return hiveSyncConfig;
}
}

View File

@@ -290,6 +290,7 @@ object DataSourceWriteOptions {
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = "hoodie.datasource.hive_sync.assume_date_partitioning"
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.use_pre_apache_input_format"
val HIVE_USE_JDBC_OPT_KEY = "hoodie.datasource.hive_sync.use_jdbc"
val HIVE_SKIP_RO_SUFFIX = "hoodie.datasource.hive_sync.skip_ro_suffix"
// DEFAULT FOR HIVE SPECIFIC CONFIGS
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
@@ -305,6 +306,7 @@ object DataSourceWriteOptions {
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false"
val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false"
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = "true"
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = "false"
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable"

View File

@@ -171,6 +171,8 @@ public class HiveTestService {
Files.createTempDirectory(System.currentTimeMillis() + "-").toFile().getAbsolutePath());
conf.set("datanucleus.schema.autoCreateTables", "true");
conf.set("hive.metastore.schema.verification", "false");
conf.set("datanucleus.autoCreateSchema", "true");
conf.set("datanucleus.fixedDatastore", "false");
setSystemProperty("derby.stream.error.file", derbyLogFile.getPath());
return new HiveConf(conf, this.getClass());

View File

@@ -53,9 +53,10 @@ public class FilebasedSchemaProvider extends SchemaProvider {
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());
String sourceFile = props.getString(Config.SOURCE_SCHEMA_FILE_PROP);
this.fs = FSUtils.getFs(sourceFile, jssc.hadoopConfiguration(), true);
try {
this.sourceSchema = new Schema.Parser().parse(fs.open(new Path(props.getString(Config.SOURCE_SCHEMA_FILE_PROP))));
this.sourceSchema = new Schema.Parser().parse(this.fs.open(new Path(sourceFile)));
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))));

View File

@@ -96,7 +96,7 @@ public class UtilitiesTestBase {
Logger rootLogger = Logger.getRootLogger();
rootLogger.setLevel(Level.ERROR);
Logger.getLogger("org.apache.spark").setLevel(Level.WARN);
initClass(false);
initClass(true);
}
public static void initClass(boolean startHiveService) throws Exception {

View File

@@ -6,9 +6,7 @@
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.
@@ -16,7 +14,7 @@
limitations under the License.
-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>hudi</artifactId>
<groupId>org.apache.hudi</groupId>
@@ -69,42 +67,89 @@
<include>commons-dbcp:commons-dbcp</include>
<include>commons-lang:commons-lang</include>
<include>commons-pool:commons-pool</include>
<include>org.apache.hudi:hudi-common</include>
<include>org.apache.hudi:hudi-client</include>
<include>org.apache.hudi:hudi-utilities_${scala.binary.version}</include>
<include>org.apache.hudi:hudi-spark_${scala.binary.version}</include>
<include>org.apache.hudi:hudi-hive-sync</include>
<include>org.apache.hudi:hudi-sync-common</include>
<include>org.apache.hudi:hudi-hadoop-mr</include>
<include>org.apache.hudi:hudi-timeline-service</include>
<include>org.apache.hudi:hudi-integ-test</include>
<include>org.jetbrains.kotlin:kotlin-stdlib-jdk8</include>
<include>org.jetbrains.kotlin:kotlin-stdlib</include>
<include>org.jetbrains.kotlin:kotlin-stdlib-common</include>
<include>org.jetbrains:annotations</include>
<include>org.jetbrains.kotlin:kotlin-stdlib-jdk7</include>
<include>org.eclipse.jetty:jetty-server</include>
<include>org.eclipse.jetty:jetty-http</include>
<include>org.eclipse.jetty:jetty-util</include>
<include>org.eclipse.jetty:jetty-io</include>
<include>org.eclipse.jetty:jetty-webapp</include>
<include>org.eclipse.jetty:jetty-xml</include>
<include>org.eclipse.jetty:jetty-servlet</include>
<include>org.eclipse.jetty:jetty-security</include>
<include>org.eclipse.jetty.websocket:websocket-server</include>
<include>org.eclipse.jetty.websocket:websocket-common</include>
<include>org.eclipse.jetty.websocket:websocket-api</include>
<include>org.eclipse.jetty.websocket:websocket-client</include>
<include>org.eclipse.jetty:jetty-client</include>
<include>org.eclipse.jetty.websocket:websocket-servlet</include>
<include>org.mortbay.jetty:jetty</include>
<include>org.mortbay.jetty:jetty-util</include>
<include>org.rocksdb:rocksdbjni</include>
<include>com.beust:jcommander</include>
<include>com.twitter:bijection-avro_${scala.binary.version}</include>
<include>com.twitter:bijection-core_${scala.binary.version}</include>
<include>org.apache.parquet:parquet-avro</include>
<include>com.twitter:parquet-avro</include>
<include>com.twitter.common:objectsize</include>
<include>io.confluent:kafka-avro-serializer</include>
<include>io.confluent:common-config</include>
<include>io.confluent:common-utils</include>
<include>io.confluent:kafka-schema-registry-client</include>
<include>org.apache.kafka:kafka_${scala.binary.version}</include>
<include>org.apache.kafka:kafka-clients</include>
<include>io.dropwizard.metrics:metrics-core</include>
<include>io.dropwizard.metrics:metrics-graphite</include>
<include>io.javalin:javalin</include>
<include>org.apache.spark:spark-streaming-kafka-0-10_${scala.binary.version}</include>
<include>org.apache.kafka:kafka_${scala.binary.version}</include>
<include>com.101tec:zkclient</include>
<include>org.apache.kafka:kafka-clients</include>
<include>org.apache.hive:hive-common</include>
<include>org.apache.hive:hive-service</include>
<include>org.apache.hive:hive-metastore</include>
<include>org.apache.hive:hive-jdbc</include>
<include>org.apache.hive:hive-exec</include>
<include>com.esotericsoftware:kryo-shaded</include>
<include>org.objenesis:objenesis</include>
<include>com.esotericsoftware:minlog</include>
<include>com.yammer.metrics:metrics-core</include>
<include>org.apache.thrift:libfb303</include>
<include>org.apache.thrift:libthrift</include>
<include>org.apache.httpcomponents:httpclient</include>
<include>org.apache.httpcomponents:httpcore</include>
<include>org.apache.httpcomponents:fluent-hc</include>
<include>org.codehaus.jackson:jackson-core-asl</include>
<include>org.codehaus.jackson:jackson-mapper-asl</include>
<include>com.fasterxml.jackson.core:jackson-annotations</include>
<include>com.fasterxml.jackson.core:jackson-core</include>
<include>com.fasterxml.jackson.core:jackson-databind</include>
<include>com.fasterxml.jackson.dataformat:jackson-dataformat-yaml</include>
<include>org.apache.htrace:htrace-core</include>
</includes>
</artifactSet>
<relocations>
@@ -245,7 +290,7 @@
<dependency>
<groupId>io.javalin</groupId>
<artifactId>javalin</artifactId>
<version>2.4.0</version>
<version>2.8.0</version>
</dependency>
<dependency>
@@ -276,7 +321,7 @@
<version>${project.version}</version>
<classifier>tests</classifier>
<type>test-jar</type>
<scope>test</scope>
<scope>compile</scope>
</dependency>
<dependency>
@@ -322,7 +367,14 @@
<groupId>${hive.groupid}</groupId>
<artifactId>hive-exec</artifactId>
<version>${hive.version}</version>
<classifier>${hive.exec.classifier}</classifier>
<scope>compile</scope>
</dependency>
<dependency>
<groupId>${hive.groupid}</groupId>
<artifactId>hive-metastore</artifactId>
<version>${hive.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
@@ -387,6 +439,7 @@
<groupId>org.apache.hudi</groupId>
<artifactId>hudi-utilities_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>compile</scope>
</dependency>
<dependency>
@@ -562,4 +615,3 @@
</dependencies>
</project>