Test Suite should work with Docker + Unit Tests
This commit is contained in:
@@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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();
|
||||
}
|
||||
|
||||
|
||||
@@ -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());
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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:
|
||||
|
||||
@@ -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);
|
||||
|
||||
@@ -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 {
|
||||
|
||||
@@ -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,
|
||||
|
||||
40
hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml
Normal file
40
hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml
Normal 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
|
||||
40
hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml
Normal file
40
hudi-integ-test/src/test/resources/unit-test-mor-dag.yaml
Normal 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
|
||||
Reference in New Issue
Block a user