1
0

[HUDI-394] Provide a basic implementation of test suite

This commit is contained in:
Nishith Agarwal
2019-11-01 12:40:09 -07:00
committed by n3nash
parent d5b593b7d9
commit 2fc2b01d86
102 changed files with 8633 additions and 64 deletions

View File

@@ -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.integ.testsuite;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.Iterator;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.integ.testsuite.configuration.DFSDeltaConfig;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
import org.apache.hudi.integ.testsuite.generator.FlexibleSchemaRecordGenerationIterator;
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
import org.apache.hudi.integ.testsuite.utils.TestUtils;
import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
import org.apache.hudi.integ.testsuite.writer.DFSDeltaWriterAdapter;
import org.apache.hudi.integ.testsuite.writer.DeltaInputWriter;
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
import org.apache.hudi.integ.testsuite.writer.DeltaWriterAdapter;
import org.apache.hudi.integ.testsuite.writer.DeltaWriterFactory;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.mockito.Mockito;
public class TestDFSHoodieTestSuiteWriterAdapter extends UtilitiesTestBase {
private FilebasedSchemaProvider schemaProvider;
private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/";
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterAll
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@BeforeEach
public void setup() throws Exception {
super.setup();
schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFSWithAbsoluteScope(
System.getProperty("user.dir") + "/.." + COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH,
"complex-source.avsc"), jsc);
}
@AfterEach
public void teardown() throws Exception {
super.teardown();
}
@Test
public void testDFSOneFileWrite() throws IOException {
DeltaInputWriter<GenericRecord> mockFileSinkWriter = Mockito.mock(AvroFileDeltaInputWriter.class);
DeltaWriteStats mockDeltaWriteStats = Mockito.mock(DeltaWriteStats.class);
when(mockFileSinkWriter.getNewWriter()).thenReturn(mockFileSinkWriter);
when(mockFileSinkWriter.canWrite()).thenReturn(true);
when(mockFileSinkWriter.getDeltaWriteStats()).thenReturn(mockDeltaWriteStats);
DeltaWriterAdapter<GenericRecord> dfsDeltaWriterAdapter = new DFSDeltaWriterAdapter(mockFileSinkWriter);
JavaRDD<GenericRecord> records = TestUtils.makeRDD(jsc, 10);
dfsDeltaWriterAdapter.write(records.collect().iterator());
Mockito.verify(mockFileSinkWriter, times(10)).canWrite();
Mockito.verify(mockFileSinkWriter, times(1)).close();
}
@Test
public void testDFSTwoFilesWriteWithRollover() throws IOException {
DeltaInputWriter<GenericRecord> mockFileSinkWriter = Mockito.mock(AvroFileDeltaInputWriter.class);
DeltaWriteStats mockDeltaWriteStats = Mockito.mock(DeltaWriteStats.class);
when(mockFileSinkWriter.getNewWriter()).thenReturn(mockFileSinkWriter);
when(mockFileSinkWriter.canWrite()).thenReturn(false, true);
when(mockFileSinkWriter.getDeltaWriteStats()).thenReturn(mockDeltaWriteStats);
DeltaWriterAdapter<GenericRecord> dfsDeltaWriterAdapter = new DFSDeltaWriterAdapter(mockFileSinkWriter);
Iterator<GenericRecord> mockIterator = Mockito.mock(Iterator.class);
when(mockIterator.hasNext()).thenReturn(true, true, true, false);
dfsDeltaWriterAdapter.write(mockIterator);
Mockito.verify(mockFileSinkWriter, times(2)).canWrite();
Mockito.verify(mockFileSinkWriter, times(1)).getNewWriter();
Mockito.verify(mockFileSinkWriter, times(2)).close();
}
@Test
public void testDFSWorkloadSinkWithMultipleFilesFunctional() throws IOException {
DeltaConfig dfsSinkConfig = new DFSDeltaConfig(DeltaOutputMode.DFS, DeltaInputType.AVRO,
new SerializableConfiguration(jsc.hadoopConfiguration()), dfsBasePath, dfsBasePath,
schemaProvider.getSourceSchema().toString(), 10240L);
DeltaWriterAdapter<GenericRecord> dfsDeltaWriterAdapter = DeltaWriterFactory
.getDeltaWriterAdapter(dfsSinkConfig, 1);
FlexibleSchemaRecordGenerationIterator itr = new FlexibleSchemaRecordGenerationIterator(1000,
schemaProvider.getSourceSchema().toString());
dfsDeltaWriterAdapter.write(itr);
FileSystem fs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
FileStatus[] fileStatuses = fs.listStatus(new Path(dfsBasePath));
// Since maxFileSize was 10240L and we produced 1K records each close to 1K size, we should produce more than
// 1 file
assertTrue(fileStatuses.length > 0);
}
}

View File

@@ -0,0 +1,147 @@
/*
* 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;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.integ.testsuite.reader.SparkBasedReader;
import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
import org.apache.hudi.integ.testsuite.writer.DeltaInputWriter;
import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
import org.apache.hudi.integ.testsuite.generator.GenericRecordFullPayloadGenerator;
import org.apache.hudi.integ.testsuite.reader.SparkBasedReader;
import org.apache.hudi.integ.testsuite.writer.AvroFileDeltaInputWriter;
import org.apache.hudi.integ.testsuite.writer.DeltaInputWriter;
import org.apache.hudi.integ.testsuite.writer.DeltaWriteStats;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
public class TestFileDeltaInputWriter extends UtilitiesTestBase {
private FilebasedSchemaProvider schemaProvider;
private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/";
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterAll
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@BeforeEach
public void setup() throws Exception {
super.setup();
schemaProvider = new FilebasedSchemaProvider(Helpers.setupSchemaOnDFSWithAbsoluteScope(System.getProperty("user.dir") + "/.."
+ COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH, "complex-source.avsc"), jsc);
}
@AfterEach
public void teardown() throws Exception {
super.teardown();
}
@Test
public void testAvroFileSinkWriter() throws IOException {
// 1. Create a Avro File Sink Writer
DeltaInputWriter<GenericRecord> fileSinkWriter =
new AvroFileDeltaInputWriter(jsc.hadoopConfiguration(), dfsBasePath + "/input", schemaProvider.getSourceSchema()
.toString(), 1024 * 1024L);
GenericRecordFullPayloadGenerator payloadGenerator =
new GenericRecordFullPayloadGenerator(schemaProvider.getSourceSchema());
// 2. Generate 100 avro payloads and write them to an avro file
IntStream.range(0, 100).forEach(a -> {
try {
fileSinkWriter.writeData(payloadGenerator.getNewPayload());
} catch (IOException io) {
throw new UncheckedIOException(io);
}
});
fileSinkWriter.close();
DeltaWriteStats deltaWriteStats = fileSinkWriter.getDeltaWriteStats();
FileSystem fs = FSUtils.getFs(dfsBasePath, jsc.hadoopConfiguration());
FileStatus[] fileStatuses = fs.listStatus(new Path(deltaWriteStats.getFilePath()));
// Atleast 1 file was written
assertEquals(1, fileStatuses.length);
// File length should be greater than 0
assertTrue(fileStatuses[0].getLen() > 0);
// File length should be the same as the number of bytes written
assertTrue(deltaWriteStats.getBytesWritten() > 0);
List<String> paths = Arrays.asList(fs.globStatus(new Path(dfsBasePath + "/*/*.avro")))
.stream().map(f -> f.getPath().toString()).collect(Collectors.toList());
JavaRDD<GenericRecord> writtenRecords =
SparkBasedReader.readAvro(sparkSession, schemaProvider.getSourceSchema().toString(), paths, Option.empty(),
Option.empty());
// Number of records written should be 100
assertEquals(writtenRecords.count(), 100);
// Number of records in file should match with the stats
assertEquals(writtenRecords.count(), deltaWriteStats.getRecordsWritten());
}
@Test
public void testAvroFileSinkCreateNewWriter() throws IOException {
// 1. Create a Avro File Sink Writer
DeltaInputWriter<GenericRecord> fileSinkWriter =
new AvroFileDeltaInputWriter(jsc.hadoopConfiguration(), dfsBasePath,
schemaProvider.getSourceSchema().toString(),
1024 * 1024L);
GenericRecordFullPayloadGenerator payloadGenerator =
new GenericRecordFullPayloadGenerator(schemaProvider.getSourceSchema());
// 2. Generate 100 avro payloads and write them to an avro file
IntStream.range(0, 100).forEach(a -> {
try {
fileSinkWriter.writeData(payloadGenerator.getNewPayload());
} catch (IOException io) {
throw new UncheckedIOException(io);
}
});
fileSinkWriter.close();
String oldFilePath = fileSinkWriter.getDeltaWriteStats().getFilePath();
assertFalse(oldFilePath == null);
DeltaInputWriter<GenericRecord> newFileSinkWriter = fileSinkWriter.getNewWriter();
newFileSinkWriter.close();
DeltaWriteStats newStats = newFileSinkWriter.getDeltaWriteStats();
assertEquals(newStats.getBytesWritten(), 3674);
assertEquals(newStats.getRecordsWritten(), 0);
assertTrue(newStats.getFilePath() != null);
}
}

View File

@@ -0,0 +1,80 @@
/*
* 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.configuration;
import static junit.framework.Assert.assertTrue;
import static junit.framework.TestCase.assertEquals;
import java.util.ArrayList;
import java.util.List;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.WorkflowDag;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
import org.junit.jupiter.api.Test;
public class TestWorkflowBuilder {
@Test
public void testWorkloadOperationSequenceBuilder() {
DagNode root = new InsertNode(DeltaConfig.Config.newBuilder()
.withNumRecordsToInsert(10000)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(2)
.withRecordSize(1000).build());
DagNode child1 = new UpsertNode(DeltaConfig.Config.newBuilder()
.withNumRecordsToUpdate(10000)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(2)
.withRecordSize(1000).build());
root.addChildNode(child1);
child1.addParentNode(root);
List<DagNode> rootNodes = new ArrayList<>();
rootNodes.add(root);
WorkflowDag workflowDag = new WorkflowDag(rootNodes);
assertEquals(workflowDag.getNodeList().size(), 1);
assertEquals(((DagNode) workflowDag.getNodeList().get(0)).getChildNodes().size(), 1);
DagNode dagNode = (DagNode) workflowDag.getNodeList().get(0);
assertTrue(dagNode instanceof InsertNode);
DeltaConfig.Config config = dagNode.getConfig();
assertEquals(config.getNumInsertPartitions(), 1);
assertEquals(config.getRecordSize(), 1000);
assertEquals(config.getRepeatCount(), 2);
assertEquals(config.getNumRecordsInsert(), 10000);
assertEquals(config.getNumRecordsUpsert(), 0);
dagNode = (DagNode) ((DagNode) workflowDag.getNodeList().get(0)).getChildNodes().get(0);
assertTrue(dagNode instanceof UpsertNode);
config = dagNode.getConfig();
assertEquals(config.getNumInsertPartitions(), 1);
assertEquals(config.getRecordSize(), 1000);
assertEquals(config.getRepeatCount(), 2);
assertEquals(config.getNumRecordsInsert(), 0);
assertEquals(config.getNumRecordsUpsert(), 10000);
}
}

View File

@@ -0,0 +1,93 @@
/*
* 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.converter;
import static junit.framework.TestCase.assertTrue;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.integ.testsuite.utils.TestUtils;
import org.apache.hudi.integ.testsuite.utils.TestUtils;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import scala.Tuple2;
public class TestUpdateConverter {
private JavaSparkContext jsc;
@BeforeEach
public void setup() throws Exception {
jsc = UtilHelpers.buildSparkContext(this.getClass().getName() + "-hoodie", "local[1]");
}
@AfterEach
public void teardown() {
jsc.stop();
}
@Test
public void testGenerateUpdateRecordsFromInputRecords() throws Exception {
JavaRDD<GenericRecord> inputRDD = TestUtils.makeRDD(jsc, 10);
String schemaStr = inputRDD.take(1).get(0).getSchema().toString();
int minPayloadSize = 1000;
// 2. DFS converter reads existing records and generates random updates for the same row keys
UpdateConverter updateConverter = new UpdateConverter(schemaStr, minPayloadSize,
Arrays.asList("timestamp"), Arrays.asList("_row_key"));
List<String> insertRowKeys = inputRDD.map(r -> r.get("_row_key").toString()).collect();
assertTrue(inputRDD.count() == 10);
JavaRDD<GenericRecord> outputRDD = updateConverter.convert(inputRDD);
List<String> updateRowKeys = outputRDD.map(row -> row.get("_row_key").toString()).collect();
// The insert row keys should be the same as update row keys
assertTrue(insertRowKeys.containsAll(updateRowKeys));
Map<String, GenericRecord> inputRecords = inputRDD.mapToPair(r -> new Tuple2<>(r.get("_row_key").toString(), r))
.collectAsMap();
List<GenericRecord> updateRecords = outputRDD.collect();
updateRecords.stream().forEach(updateRecord -> {
GenericRecord inputRecord = inputRecords.get(updateRecord.get("_row_key").toString());
assertTrue(areRecordsDifferent(inputRecord, updateRecord));
});
}
/**
* Checks if even a single field in the 2 records is different (except the row key which is the same for an update).
*/
private boolean areRecordsDifferent(GenericRecord in, GenericRecord up) {
for (Field field : in.getSchema().getFields()) {
if (field.name() == "_row_key") {
continue;
} else {
// Just convert all types to string for now since all are primitive
if (in.get(field.name()).toString() != up.get(field.name()).toString()) {
return true;
}
}
}
return false;
}
}

View File

@@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.integ.testsuite.dag;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Function;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
import org.apache.hudi.integ.testsuite.dag.nodes.ValidateNode;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.integ.testsuite.dag.nodes.UpsertNode;
import org.apache.hudi.integ.testsuite.dag.nodes.ValidateNode;
import org.apache.spark.api.java.JavaRDD;
public class ComplexDagGenerator implements WorkflowDagGenerator {
@Override
public WorkflowDag build() {
DagNode root = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(1000)
.withNumInsertPartitions(3)
.withRecordSize(1000).build());
DagNode child1 = new UpsertNode(Config.newBuilder()
.withNumRecordsToUpdate(999)
.withNumRecordsToInsert(1000)
.withNumUpsertFiles(1)
.withNumUpsertPartitions(1)
.withNumInsertPartitions(1)
.withRecordSize(10000).build());
Function<List<DagNode<JavaRDD<WriteStatus>>>, Boolean> function = (dagNodes) -> {
DagNode<JavaRDD<WriteStatus>> parent1 = dagNodes.get(0);
List<WriteStatus> statuses = parent1.getResult().collect();
long totalRecordsTouched = statuses.stream().map(st -> st.getStat().getNumUpdateWrites() + st.getStat()
.getNumInserts()).reduce((a, b) -> a + b).get();
boolean b1 = totalRecordsTouched == parent1.getConfig().getNumRecordsInsert()
+ parent1.getConfig().getNumRecordsUpsert();
boolean b2 = statuses.size() > parent1.getConfig().getNumUpsertFiles();
DagNode<JavaRDD<WriteStatus>> parent2 = parent1.getParentNodes().get(0);
statuses = parent2.getResult().collect();
totalRecordsTouched = statuses.stream().map(st -> st.getStat().getNumUpdateWrites() + st.getStat()
.getNumInserts()).reduce((a, b) -> a + b).get();
boolean b3 = totalRecordsTouched == parent2.getConfig().getNumRecordsInsert()
* parent2.getConfig().getNumInsertPartitions() + parent2.getConfig().getNumRecordsUpsert();
return b1 & b2 & b3;
};
DagNode child2 = new ValidateNode(Config.newBuilder().build(), function);
root.addChildNode(child1);
// child1.addParentNode(root);
child1.addChildNode(child2);
// child2.addParentNode(child1);
List<DagNode> rootNodes = new ArrayList<>();
rootNodes.add(root);
return new WorkflowDag(rootNodes);
}
}

View File

@@ -0,0 +1,58 @@
/*
* 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;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.HiveQueryNode;
import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
public class HiveSyncDagGenerator implements WorkflowDagGenerator {
@Override
public WorkflowDag build() {
DagNode root = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(100)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(1)
.withRecordSize(1000).build());
DagNode child1 = new HiveSyncNode(Config.newBuilder().withHiveLocal(true).build());
root.addChildNode(child1);
DagNode child2 = new HiveQueryNode(Config.newBuilder().withHiveLocal(true).withHiveQueryAndResults(Arrays
.asList(Pair.of("select " + "count(*) from testdb1.table1 group " + "by rider having count(*) < 1", 0)))
.build());
child1.addChildNode(child2);
List<DagNode> rootNodes = new ArrayList<>();
rootNodes.add(root);
return new WorkflowDag(rootNodes);
}
}

View File

@@ -0,0 +1,58 @@
/*
* 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;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.HiveQueryNode;
import org.apache.hudi.integ.testsuite.dag.nodes.HiveSyncNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
public class HiveSyncDagGeneratorMOR implements WorkflowDagGenerator {
@Override
public WorkflowDag build() {
DagNode root = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(100)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(1)
.withRecordSize(1000).build());
DagNode child1 = new HiveSyncNode(Config.newBuilder().withHiveLocal(true).build());
root.addChildNode(child1);
DagNode child2 = new HiveQueryNode(Config.newBuilder().withHiveLocal(true).withHiveQueryAndResults(Arrays
.asList(Pair.of("select " + "count(*) from testdb1.table1_rt group " + "by rider having count(*) < 1", 0)))
.build());
child1.addChildNode(child2);
List<DagNode> rootNodes = new ArrayList<>();
rootNodes.add(root);
return new WorkflowDag(rootNodes);
}
}

View File

@@ -0,0 +1,95 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.integ.testsuite.dag;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.util.ArrayList;
import java.util.List;
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
import org.apache.hudi.integ.testsuite.dag.nodes.DagNode;
import org.apache.hudi.integ.testsuite.dag.nodes.InsertNode;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.junit.jupiter.api.Assertions;
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";
@Test
public void testConvertDagToYaml() throws Exception {
ComplexDagGenerator dag = new ComplexDagGenerator();
String yaml = DagUtils.convertDagToYaml(dag.build());
System.out.println(yaml);
}
@Test
public void testConvertYamlToDag() throws Exception {
WorkflowDag dag = DagUtils.convertYamlToDag(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath((System.getProperty("user.dir") + "/.." + COW_DAG_DOCKER_DEMO_RELATIVE_PATH)));
assertEquals(dag.getNodeList().size(), 1);
Assertions.assertEquals(((DagNode) dag.getNodeList().get(0)).getParentNodes().size(), 0);
assertEquals(((DagNode) dag.getNodeList().get(0)).getChildNodes().size(), 1);
DagNode firstChild = (DagNode) ((DagNode) dag.getNodeList().get(0)).getChildNodes().get(0);
assertEquals(firstChild.getParentNodes().size(), 1);
assertEquals(firstChild.getChildNodes().size(), 1);
assertEquals(((DagNode) firstChild.getChildNodes().get(0)).getChildNodes().size(), 1);
}
public static class ComplexDagGenerator implements WorkflowDagGenerator {
@Override
public WorkflowDag build() {
DagNode root = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(1000000)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(2)
.withRecordSize(1000).build());
DagNode child1 = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(1000000)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(2)
.withRecordSize(1000).build());
DagNode child2 = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(1000000)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(2)
.withRecordSize(1000).build());
root.addChildNode(child1);
root.addChildNode(child2);
DagNode child3 = new InsertNode(Config.newBuilder()
.withNumRecordsToInsert(1000000)
.withNumInsertPartitions(1)
.withNumTimesToRepeat(2)
.withRecordSize(1000).build());
child2.addChildNode(child3);
List<DagNode> rootNodes = new ArrayList<>();
rootNodes.add(root);
return new WorkflowDag(rootNodes);
}
}
}

View File

@@ -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.integ.testsuite.generator;
import static junit.framework.TestCase.assertEquals;
import org.apache.avro.Schema;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.junit.jupiter.api.Test;
public class TestGenericRecordPayloadEstimator {
private static final String SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/source.avsc";
private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH =
"/docker/demo/config/test-suite/complex-source.avsc";
@Test
public void testSimpleSchemaSize() throws Exception {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
GenericRecordFullPayloadSizeEstimator estimator =
new GenericRecordFullPayloadSizeEstimator(schema);
Pair<Integer, Integer> estimateAndNumComplexFields = estimator.typeEstimateAndNumComplexFields();
assertEquals(estimateAndNumComplexFields.getRight().intValue(), 0);
assertEquals(estimateAndNumComplexFields.getLeft().intValue(), 156);
}
@Test
public void testComplexSchemaSize() throws Exception {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers.readFileFromAbsolutePath(
System.getProperty("user.dir") + "/.." + COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
GenericRecordFullPayloadSizeEstimator estimator =
new GenericRecordFullPayloadSizeEstimator(schema);
Pair<Integer, Integer> estimateAndNumComplexFields = estimator.typeEstimateAndNumComplexFields();
assertEquals(estimateAndNumComplexFields.getRight().intValue(), 1);
assertEquals(estimateAndNumComplexFields.getLeft().intValue(), 1278);
}
}

View File

@@ -0,0 +1,130 @@
/*
* 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.generator;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.stream.IntStream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.junit.jupiter.api.Test;
public class TestGenericRecordPayloadGenerator {
private static final String SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/source.avsc";
private static final String COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH =
"/docker/demo/config/test-suite/complex-source.avsc";
@Test
public void testSimplePayload() throws Exception {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
GenericRecord record = payloadGenerator.getNewPayload();
// The generated payload should validate with the provided schema
payloadGenerator.validate(record);
}
@Test
public void testComplexPayload() throws IOException {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." +
COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
GenericRecord record = payloadGenerator.getNewPayload();
// The generated payload should validate with the provided schema
assertTrue(payloadGenerator.validate(record));
}
@Test
public void testComplexPartialPayload() throws IOException {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." +
COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
GenericRecordPartialPayloadGenerator payloadGenerator = new GenericRecordPartialPayloadGenerator(schema);
IntStream.range(0, 10).forEach(a -> {
GenericRecord record = payloadGenerator.getNewPayload();
// The generated payload should validate with the provided schema
assertTrue(payloadGenerator.validate(record));
});
}
@Test
public void testUpdatePayloadGenerator() throws IOException {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema);
List<String> insertRowKeys = new ArrayList<>();
List<String> updateRowKeys = new ArrayList<>();
List<Long> insertTimeStamps = new ArrayList<>();
List<Long> updateTimeStamps = new ArrayList<>();
List<GenericRecord> records = new ArrayList<>();
// Generate 10 new records
IntStream.range(0, 10).forEach(a -> {
GenericRecord record = payloadGenerator.getNewPayload();
records.add(record);
insertRowKeys.add(record.get("_row_key").toString());
insertTimeStamps.add((Long) record.get("timestamp"));
});
List<String> blacklistFields = Arrays.asList("_row_key");
records.stream().forEach(a -> {
// Generate 10 updated records
GenericRecord record = payloadGenerator.getUpdatePayload(a, blacklistFields);
updateRowKeys.add(record.get("_row_key").toString());
updateTimeStamps.add((Long) record.get("timestamp"));
});
// The row keys from insert payloads should match all the row keys from the update payloads
assertTrue(insertRowKeys.containsAll(updateRowKeys));
// The timestamp field for the insert payloads should not all match with the update payloads
assertFalse(insertTimeStamps.containsAll(updateTimeStamps));
}
@Test
public void testSimplePayloadWithLargeMinSize() throws Exception {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." + SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
int minPayloadSize = 1000;
GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(schema,
minPayloadSize);
GenericRecord record = payloadGenerator.getNewPayload();
// The payload generated is less than minPayloadSize due to no collections present
assertTrue(HoodieAvroUtils.avroToBytes(record).length < minPayloadSize);
}
@Test
public void testComplexPayloadWithLargeMinSize() throws Exception {
Schema schema = new Schema.Parser().parse(UtilitiesTestBase.Helpers
.readFileFromAbsolutePath(System.getProperty("user.dir") + "/.." +
COMPLEX_SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH));
int minPayloadSize = 10000;
GenericRecordFullPayloadGenerator payloadGenerator = new GenericRecordFullPayloadGenerator(
schema, minPayloadSize);
GenericRecord record = payloadGenerator.getNewPayload();
// The payload generated should be within 10% extra of the minPayloadSize
assertTrue(HoodieAvroUtils.avroToBytes(record).length < minPayloadSize + 0.1 * minPayloadSize);
}
}

View File

@@ -0,0 +1,224 @@
/*
* 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.job;
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.util.UUID;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob;
import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig;
import org.apache.hudi.integ.testsuite.dag.ComplexDagGenerator;
import org.apache.hudi.integ.testsuite.dag.HiveSyncDagGenerator;
import org.apache.hudi.integ.testsuite.dag.HiveSyncDagGeneratorMOR;
import org.apache.hudi.integ.testsuite.dag.WorkflowDagGenerator;
import org.apache.hudi.integ.testsuite.reader.DeltaInputType;
import org.apache.hudi.integ.testsuite.writer.DeltaOutputMode;
import org.apache.hudi.keygen.TimestampBasedKeyGenerator;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
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.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with useDeltaStreamer={0}, tableType={1}";
private static final String BASE_PROPERTIES_DOCKER_DEMO_RELATIVE_PATH = "/docker/demo/config/test-suite/base"
+ ".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";
public static Stream<Arguments> configParams() {
Object[][] data =
new Object[][] {{false, "COPY_ON_WRITE"}};
return Stream.of(data).map(Arguments::of);
}
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
// prepare the configs.
UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+ BASE_PROPERTIES_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/base.properties");
UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+ SOURCE_SCHEMA_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/source.avsc");
UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+ 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");
UtilitiesTestBase.Helpers.copyToDFSFromAbsolutePath(System.getProperty("user.dir") + "/.."
+ MOR_DAG_DOCKER_DEMO_RELATIVE_PATH, dfs, dfsBasePath + "/complex-dag-mor.yaml");
TypedProperties props = new TypedProperties();
props.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
props.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp");
props.setProperty("hoodie.deltastreamer.keygen.timebased.timestamp.type", "UNIX_TIMESTAMP");
props.setProperty("hoodie.deltastreamer.keygen.timebased.output.dateformat", "yyyy/MM/dd");
props.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
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.write.keytranslator.class", "org.apache.hudi"
+ ".DayBasedPartitionPathKeyTranslator");
props.setProperty("hoodie.compact.inline.max.delta.commits", "3");
props.setProperty("hoodie.parquet.max.file.size", "1024000");
props.setProperty("hoodie.compact.inline.max.delta.commits", "0");
// Hive Configs
props.setProperty(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), "jdbc:hive2://127.0.0.1:9999/");
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), "testdb1");
props.setProperty(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), "table1");
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "datestr");
props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), TimestampBasedKeyGenerator.class.getName());
UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/test-source"
+ ".properties");
// Properties used for the delta-streamer which incrementally pulls from upstream DFS Avro source and
// writes to downstream hudi table
TypedProperties downstreamProps = new TypedProperties();
downstreamProps.setProperty("include", "base.properties");
downstreamProps.setProperty("hoodie.datasource.write.recordkey.field", "_row_key");
downstreamProps.setProperty("hoodie.datasource.write.partitionpath.field", "timestamp");
// Source schema is the target schema of upstream table
downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.source.schema.file", dfsBasePath + "/source.avsc");
downstreamProps.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/source.avsc");
UtilitiesTestBase.Helpers.savePropsToDFS(downstreamProps, dfs,
dfsBasePath + "/test-downstream-source.properties");
// these tests cause a lot of log verbosity from spark, turning it down
Logger.getLogger("org.apache.spark").setLevel(Level.WARN);
}
@AfterAll
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@BeforeEach
public void setup() throws Exception {
super.setup();
}
@AfterEach
public void teardown() throws Exception {
super.teardown();
}
// 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);
String inputBasePath = dfsBasePath + "/input/" + UUID.randomUUID().toString();
String outputBasePath = dfsBasePath + "/result/" + UUID.randomUUID().toString();
HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, tableType);
cfg.workloadDagGenerator = ComplexDagGenerator.class.getName();
HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
hoodieTestSuiteJob.runTestSuite();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
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);
String inputBasePath = dfsBasePath + "/input";
String outputBasePath = dfsBasePath + "/result";
HoodieTestSuiteConfig cfg = makeConfig(inputBasePath, outputBasePath, useDeltaStreamer, tableType);
if (tableType == HoodieTableType.COPY_ON_WRITE.name()) {
cfg.workloadDagGenerator = HiveSyncDagGenerator.class.getName();
} else {
cfg.workloadDagGenerator = HiveSyncDagGeneratorMOR.class.getName();
}
HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
hoodieTestSuiteJob.runTestSuite();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
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);
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";
HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
hoodieTestSuiteJob.runTestSuite();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
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);
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";
HoodieTestSuiteJob hoodieTestSuiteJob = new HoodieTestSuiteJob(cfg, jsc);
hoodieTestSuiteJob.runTestSuite();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(new Configuration(), cfg.targetBasePath);
assertEquals(metaClient.getActiveTimeline().getCommitsTimeline().getInstants().count(), 7);
}
protected HoodieTestSuiteConfig makeConfig(String inputBasePath, String outputBasePath, boolean useDeltaStream,
String tableType) {
HoodieTestSuiteConfig cfg = new HoodieTestSuiteConfig();
cfg.targetBasePath = outputBasePath;
cfg.inputBasePath = inputBasePath;
cfg.targetTableName = "table1";
cfg.tableType = tableType;
cfg.sourceClassName = AvroDFSSource.class.getName();
cfg.sourceOrderingField = "timestamp";
cfg.propsFilePath = dfsBasePath + "/test-source.properties";
cfg.outputTypeName = DeltaOutputMode.DFS.name();
cfg.inputFormatName = DeltaInputType.AVRO.name();
cfg.limitFileSize = 1024 * 1024L;
cfg.sourceLimit = 20000000;
cfg.workloadDagGenerator = WorkflowDagGenerator.class.getName();
cfg.schemaProviderClassName = FilebasedSchemaProvider.class.getName();
cfg.useDeltaStreamer = useDeltaStream;
return cfg;
}
}

View File

@@ -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.integ.testsuite.reader;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.integ.testsuite.utils.TestUtils;
import org.apache.hudi.integ.testsuite.utils.TestUtils;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
public class TestDFSAvroDeltaInputReader extends UtilitiesTestBase {
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterAll
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@BeforeEach
public void setup() throws Exception {
super.setup();
}
@Test
@Disabled
public void testDFSSinkReader() throws IOException {
FileSystem fs = FSUtils.getFs(dfsBasePath, new Configuration());
// Create 10 avro files with 10 records each
TestUtils.createAvroFiles(jsc, sparkSession, dfsBasePath, 10, 10);
FileStatus[] statuses = fs.globStatus(new Path(dfsBasePath + "/*/*.avro"));
DFSAvroDeltaInputReader reader =
new DFSAvroDeltaInputReader(sparkSession, TestUtils.getSchema().toString(), dfsBasePath, Option.empty(),
Option.empty());
assertEquals(reader.analyzeSingleFile(statuses[0].getPath().toString()), 5);
assertEquals(reader.read(100).count(), 100);
assertEquals(reader.read(1000).count(), 100);
assertEquals(reader.read(10).count(), 10);
assertTrue(reader.read(11).count() > 11);
}
}

View File

@@ -0,0 +1,119 @@
/*
* 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.reader;
import static junit.framework.TestCase.assertEquals;
import static junit.framework.TestCase.assertTrue;
import java.util.HashSet;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.HoodieWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
public class TestDFSHoodieDatasetInputReader extends UtilitiesTestBase {
@BeforeAll
public static void initClass() throws Exception {
UtilitiesTestBase.initClass();
}
@AfterAll
public static void cleanupClass() {
UtilitiesTestBase.cleanupClass();
}
@BeforeEach
public void setup() throws Exception {
super.setup();
HoodieTestUtils.init(jsc.hadoopConfiguration(), dfsBasePath);
}
@AfterEach
public void teardown() throws Exception {
super.teardown();
}
@Test
public void testSimpleHoodieDatasetReader() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, config);
String commitTime = client.startCommit();
HoodieTestDataGenerator generator = new HoodieTestDataGenerator();
// Insert 100 records across 3 partitions
List<HoodieRecord> inserts = generator.generateInserts(commitTime, 100);
JavaRDD<WriteStatus> writeStatuses = client.upsert(jsc.parallelize(inserts), commitTime);
writeStatuses.count();
DFSHoodieDatasetInputReader reader = new DFSHoodieDatasetInputReader(jsc, config.getBasePath(),
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema())).toString());
// Try to read 100 records for the same partition path and same file ID
JavaRDD<GenericRecord> records = reader.read(1, 1, 100L);
assertTrue(records.count() <= 100);
assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).collect()).size(),
1);
assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.FILENAME_METADATA_FIELD)).collect()).size(),
1);
// Try to read 100 records for 3 partition paths and 3 different file ids
records = reader.read(3, 3, 100L);
assertTrue(records.count() <= 100);
assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).collect()).size(),
3);
assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.FILENAME_METADATA_FIELD)).collect()).size(),
3);
// Try to read 100 records for 3 partition paths and 50% records from each file
records = reader.read(3, 3, 0.5);
assertTrue(records.count() <= 100);
assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).collect()).size(),
3);
assertEquals(new HashSet<>(records.map(p -> p.get(HoodieRecord.FILENAME_METADATA_FIELD)).collect()).size(),
3);
}
private HoodieWriteConfig makeHoodieClientConfig() throws Exception {
return makeHoodieClientConfigBuilder().build();
}
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() throws Exception {
// Prepare the AvroParquetIO
return HoodieWriteConfig.newBuilder().withPath(dfsBasePath)
.withParallelism(2, 2)
.withSchema(HoodieTestDataGenerator
.TRIP_EXAMPLE_SCHEMA);
}
}

View File

@@ -0,0 +1,63 @@
/*
* 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.utils;
import java.util.List;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.utilities.schema.RowBasedSchemaProvider;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SparkSession;
public class TestUtils {
/**
* Create a RDD of generic records for testing purposes.
*/
public static JavaRDD<GenericRecord> makeRDD(JavaSparkContext jsc, int numRecords) {
return jsc.parallelize(generateGenericRecords(numRecords));
}
/**
* Generate generic records.
*/
public static List<GenericRecord> generateGenericRecords(int numRecords) {
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
return dataGenerator.generateGenericRecords(numRecords);
}
public static void createAvroFiles(JavaSparkContext jsc, SparkSession sparkSession, String basePath, int numFiles,
int numRecordsPerFile) {
Schema schema = HoodieTestDataGenerator.AVRO_SCHEMA;
for (int i = 0; i < numFiles; i++) {
JavaRDD<GenericRecord> rdd = makeRDD(jsc, numRecordsPerFile);
AvroConversionUtils.createDataFrame(rdd.rdd(), schema.toString(), sparkSession).write()
.format("avro").option("recordName", RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME)
.option("recordNamespace", RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE).save(basePath + "/" + i);
}
}
public static Schema getSchema() {
return HoodieTestDataGenerator.AVRO_SCHEMA;
}
}

View File

@@ -15,8 +15,9 @@
# See the License for the specific language governing permissions and
# limitations under the License.
###
log4j.rootLogger=WARN, CONSOLE
log4j.logger.org.apache.hudi=DEBUG
log4j.rootLogger=ERROR, CONSOLE
log4j.logger.org.apache.hudi=ERROR
log4j.category.org.apache.spark=ERROR
# CONSOLE is set to be a ConsoleAppender.
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
@@ -25,5 +26,5 @@ log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
log4j.appender.CONSOLE.filter.a.LevelMin=ERROR
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL