nsivabalan commented on a change in pull request #2168: URL: https://github.com/apache/hudi/pull/2168#discussion_r528234217
########## File path: docker/demo/config/test-suite/cow-per-round-mixed-validate.yaml ########## @@ -0,0 +1,65 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +first_insert: Review comment: I haven't fixed these dags yet for long running test-suite. Once we have consensus that things are looking good, I will update all dags. ########## File path: hudi-integ-test/README.md ########## @@ -214,7 +216,7 @@ spark-submit \ --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 \ +--source-ordering-field test_suite_source_ordering_field \ Review comment: as per our discussion, I have added this new field from within the code. So essentially this field will be appended to both source schema and target schema and will be used for source ordering. batch_id is used as values. ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/scheduler/DagScheduler.java ########## @@ -77,33 +79,47 @@ public void schedule() throws Exception { * Method to start executing the nodes in workflow DAGs. * * @param service ExecutorService - * @param nodes Nodes to be executed + * @param workflowDag instance of workflow dag that needs to be executed * @throws Exception will be thrown if ant error occurred */ - private void execute(ExecutorService service, List<DagNode> nodes) throws Exception { + private void execute(ExecutorService service, WorkflowDag workflowDag) throws Exception { Review comment: I didn't make a whole lot of change from last time. I exposed the global params in workflowDag and using it wherever required. like rounds, delay etc. Felt this is simple and achieves our goal. ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/schema/TestSuiteFileBasedSchemaProvider.java ########## @@ -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.schema; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.integ.testsuite.dag.WriterContext; +import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; + +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Type; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class TestSuiteFileBasedSchemaProvider extends FilebasedSchemaProvider { Review comment: Added this schemaProvider to append the source ordering field to source and target schema. ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java ########## @@ -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.dag.nodes; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Tuple2; +import scala.collection.JavaConversions; +import scala.collection.JavaConverters; + +/** + * This nodes validates contents from input path are in tact with Hudi. This nodes uses spark datasource for comparison purposes. By default no configs are required for this node. But there is an + * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. + * README has more details under docker set up for usages of this node. + */ +public class ValidateDatasetNode extends DagNode<Boolean> { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + + public ValidateDatasetNode(Config config) { + this.config = config; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + + SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); + + // todo: Fix partitioning schemes. For now, assumes data based partitioning. + String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; + String hudiPath = context.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/*/*/*"; + log.warn("ValidateDataset Node: Input path " + inputPath + ", hudi path " + hudiPath); + // listing batches to be validated + String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); + } + + String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); + String partitionPathField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY()); + // todo: fix hard coded fields from configs. + // read input and resolve insert, updates, etc. + Dataset<Row> inputDf = session.read().format("avro").load(inputPath); + ExpressionEncoder encoder = getEncoder(inputDf.schema()); + Dataset<Row> inputSnapshotDf = inputDf.groupByKey( + (MapFunction<Row, String>) value -> partitionPathField + "+" + recordKeyField, Encoders.STRING()) + .reduceGroups((ReduceFunction<Row>) (v1, v2) -> { + int ts1 = v1.getAs(SchemaUtils.SOURCE_ORDERING_FIELD); + int ts2 = v2.getAs(SchemaUtils.SOURCE_ORDERING_FIELD); + if (ts1 > ts2) { + return v1; + } else { + return v2; + } + }) + .map((MapFunction<Tuple2<String, Row>, Row>) value -> value._2, encoder); + + // read from hudi and remove meta columns. + Dataset<Row> hudiDf = session.read().format("hudi").load(hudiPath); + Dataset<Row> trimmedDf = hudiDf.drop(HoodieRecord.COMMIT_TIME_METADATA_FIELD).drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD).drop(HoodieRecord.RECORD_KEY_METADATA_FIELD) + .drop(HoodieRecord.PARTITION_PATH_METADATA_FIELD).drop(HoodieRecord.FILENAME_METADATA_FIELD); + + Dataset<Row> intersectionDf = inputSnapshotDf.intersect(trimmedDf); + // the intersected df should be same as inputDf. if not, there is some mismatch. + if (inputSnapshotDf.except(intersectionDf).count() != 0) { + log.error("Data set validation failed. Total count in hudi " + trimmedDf.count() + ", input df count " + inputSnapshotDf.count()); + throw new AssertionError("Hudi contents does not match contents input data. "); + } + + String database = context.getWriterContext().getProps().getString("hoodie.datasource.hive_sync.database"); + String tableName = context.getWriterContext().getProps().getString("hoodie.datasource.hive_sync.table"); Review comment: again, couldn't find a better way to fetch db and table name. LMK if there are other better ways ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java ########## @@ -62,15 +71,38 @@ public static WorkflowDag convertYamlPathToDag(FileSystem fs, String path) throw * Converts a YAML representation to {@link WorkflowDag}. */ public static WorkflowDag convertYamlToDag(String yaml) throws IOException { + int dagRounds = DEFAULT_DAG_ROUNDS; + int intermittentDelayMins = DEFAULT_INTERMITTENT_DELAY_MINS; + String dagName = DEFAULT_DAG_NAME; Map<String, DagNode> allNodes = new HashMap<>(); final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory()); final JsonNode jsonNode = yamlReader.readTree(yaml); Iterator<Entry<String, JsonNode>> itr = jsonNode.fields(); while (itr.hasNext()) { Entry<String, JsonNode> dagNode = itr.next(); - allNodes.put(dagNode.getKey(), convertJsonToDagNode(allNodes, dagNode.getKey(), dagNode.getValue())); + String key = dagNode.getKey(); + switch (key) { + case DAG_NAME: Review comment: high level config param parsing. ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/DagUtils.java ########## @@ -62,15 +71,38 @@ public static WorkflowDag convertYamlPathToDag(FileSystem fs, String path) throw * Converts a YAML representation to {@link WorkflowDag}. */ public static WorkflowDag convertYamlToDag(String yaml) throws IOException { + int dagRounds = DEFAULT_DAG_ROUNDS; + int intermittentDelayMins = DEFAULT_INTERMITTENT_DELAY_MINS; + String dagName = DEFAULT_DAG_NAME; Map<String, DagNode> allNodes = new HashMap<>(); final ObjectMapper yamlReader = new ObjectMapper(new YAMLFactory()); final JsonNode jsonNode = yamlReader.readTree(yaml); Iterator<Entry<String, JsonNode>> itr = jsonNode.fields(); while (itr.hasNext()) { Entry<String, JsonNode> dagNode = itr.next(); - allNodes.put(dagNode.getKey(), convertJsonToDagNode(allNodes, dagNode.getKey(), dagNode.getValue())); + String key = dagNode.getKey(); + switch (key) { + case DAG_NAME: + dagName = dagNode.getValue().asText(); + break; + case DAG_ROUNDS: + dagRounds = dagNode.getValue().asInt(); + break; + case DAG_INTERMITTENT_DELAY_MINS: + intermittentDelayMins = dagNode.getValue().asInt(); + break; + case DAG_CONTENT: Review comment: as agreed, all node contents go into "dag_content" ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/writer/DFSDeltaWriterAdapter.java ########## @@ -30,22 +32,29 @@ */ public class DFSDeltaWriterAdapter implements DeltaWriterAdapter<GenericRecord> { - private DeltaInputWriter deltaInputGenerator; + private DeltaInputWriter deltaInputWriter; private List<DeltaWriteStats> metrics = new ArrayList<>(); + private int preCombineFieldVal = 0; + + public DFSDeltaWriterAdapter(DeltaInputWriter<GenericRecord> deltaInputWriter, int preCombineFieldVal) { + this.deltaInputWriter = deltaInputWriter; + this.preCombineFieldVal = preCombineFieldVal; + } - public DFSDeltaWriterAdapter(DeltaInputWriter<GenericRecord> deltaInputGenerator) { - this.deltaInputGenerator = deltaInputGenerator; + public DFSDeltaWriterAdapter(DeltaInputWriter<GenericRecord> deltaInputWriter) { + this.deltaInputWriter = deltaInputWriter; } @Override public List<DeltaWriteStats> write(Iterator<GenericRecord> input) throws IOException { while (input.hasNext()) { GenericRecord next = input.next(); - if (this.deltaInputGenerator.canWrite()) { - this.deltaInputGenerator.writeData(next); - } else if (input.hasNext()) { + next.put(SchemaUtils.SOURCE_ORDERING_FIELD, preCombineFieldVal); Review comment: this is where the source ordering field is set. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java ########## @@ -46,9 +46,9 @@ private final FileSystem fs; - private final Schema sourceSchema; + protected Schema sourceSchema; Review comment: since I am introducing a test schema provider to append source ordering field, I had to make this protected since the test schema provider extends from this. ########## File path: hudi-integ-test/src/test/resources/unit-test-cow-dag.yaml ########## @@ -13,58 +13,62 @@ # 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_partitions_insert: 1 - repeat_count: 2 - num_records_insert: 100 - type: InsertNode - deps: none -second_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_insert -first_rollback: - config: - deps: second_insert - type: RollbackNode -third_insert: - config: - record_size: 70000 - num_partitions_insert: 1 - repeat_count: 1 - num_records_insert: 100 - type: InsertNode - deps: first_rollback -first_upsert: - config: - record_size: 70000 - num_partitions_upsert: 1 - repeat_count: 1 - num_records_upsert: 100 - type: UpsertNode - deps: third_insert -first_hive_sync: - config: - queue_name: "adhoc" - engine: "mr" - type: HiveSyncNode - deps: first_upsert -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: - query1: "select count(*) from testdb1.table1" - result1: 300 - query2: "select count(*) from testdb1.table1 group by `_row_key` having count(*) > 1" - result2: 0 - type: HiveQueryNode - deps: first_hive_sync \ No newline at end of file +dag_name: unit-test-cow-dag Review comment: Here is the example of how the dag looks like w/ global params. I have tested this dag for few rounds of validation. ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java ########## @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.integ.testsuite.dag.nodes; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Tuple2; +import scala.collection.JavaConversions; +import scala.collection.JavaConverters; + +/** + * This nodes validates contents from input path are in tact with Hudi. This nodes uses spark datasource for comparison purposes. By default no configs are required for this node. But there is an + * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. + * README has more details under docker set up for usages of this node. + */ +public class ValidateDatasetNode extends DagNode<Boolean> { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + + public ValidateDatasetNode(Config config) { + this.config = config; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + + SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); + + // todo: Fix partitioning schemes. For now, assumes data based partitioning. + String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; + String hudiPath = context.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/*/*/*"; + log.warn("ValidateDataset Node: Input path " + inputPath + ", hudi path " + hudiPath); + // listing batches to be validated + String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); + } + + // todo: fix hard coded fields from configs. + // read input and resolve insert, updates, etc. + Dataset<Row> inputDf = session.read().format("avro").load(inputPath); + ExpressionEncoder encoder = getEncoder(inputDf.schema()); + Dataset<Row> inputSnapshotDf = inputDf.groupByKey( + (MapFunction<Row, String>) value -> value.getAs("timestamp") + "+" + value.getAs("_row_key"), Encoders.STRING()) Review comment: may I know where or how to fetch the record key and partition path field. I don't see any thing exposed in deltastreamer config. as of now, I am hardcoding it here. ########## File path: hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ValidateDatasetNode.java ########## @@ -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.dag.nodes; + +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; +import org.apache.hudi.integ.testsuite.dag.ExecutionContext; +import org.apache.hudi.integ.testsuite.schema.SchemaUtils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$; +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder; +import org.apache.spark.sql.catalyst.encoders.RowEncoder; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Tuple2; +import scala.collection.JavaConversions; +import scala.collection.JavaConverters; + +/** + * This nodes validates contents from input path are in tact with Hudi. This nodes uses spark datasource for comparison purposes. By default no configs are required for this node. But there is an + * optional config "delete_input_data" that you can set for this node. If set, once validation completes, contents from inputPath are deleted. This will come in handy for long running test suites. + * README has more details under docker set up for usages of this node. + */ +public class ValidateDatasetNode extends DagNode<Boolean> { + + private static Logger log = LoggerFactory.getLogger(ValidateDatasetNode.class); + + public ValidateDatasetNode(Config config) { + this.config = config; + } + + @Override + public void execute(ExecutionContext context) throws Exception { + + SparkSession session = SparkSession.builder().sparkContext(context.getJsc().sc()).getOrCreate(); + + // todo: Fix partitioning schemes. For now, assumes data based partitioning. + String inputPath = context.getHoodieTestSuiteWriter().getCfg().inputBasePath + "/*/*"; + String hudiPath = context.getHoodieTestSuiteWriter().getCfg().targetBasePath + "/*/*/*"; + log.warn("ValidateDataset Node: Input path " + inputPath + ", hudi path " + hudiPath); + // listing batches to be validated + String inputPathStr = context.getHoodieTestSuiteWriter().getCfg().inputBasePath; + FileSystem fs = new Path(inputPathStr) + .getFileSystem(context.getHoodieTestSuiteWriter().getConfiguration()); + FileStatus[] fileStatuses = fs.listStatus(new Path(inputPathStr)); + for (FileStatus fileStatus : fileStatuses) { + log.debug("Listing all Micro batches to be validated :: " + fileStatus.getPath().toString()); + } + + String recordKeyField = context.getWriterContext().getProps().getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY()); Review comment: is this the right way to fetch record key and partition path field ? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
