C0urante commented on a change in pull request #8844: URL: https://github.com/apache/kafka/pull/8844#discussion_r444445069
########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerMetricsGroupTest.java ########## @@ -0,0 +1,164 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.easymock.EasyMock.eq; +import static org.powermock.api.easymock.PowerMock.createStrictMock; +import static org.powermock.api.easymock.PowerMock.expectLastCall; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Sensor.class}) +public class WorkerMetricsGroupTest { + private final String connector = "org.FakeConnector"; + private final ConnectorTaskId task = new ConnectorTaskId(connector, 0); + private final RuntimeException exception = new RuntimeException(); + + @Test + public void testConnectorStartupRecordedMetrics() { + final WorkerMetricsGroup mockWorkerMetricsGroup = createStrictMock(WorkerMetricsGroup.class); Review comment: It's a little strange that we're mocking the class that we're testing here. Could we test on a real `WorkerMetricsGroup` object and mock its dependencies (specifically, the `ConnectMetrics` object that it takes in its constructor) instead? Might be a bit more work but would give us stronger guarantees about the accuracy and coverage of these tests. ########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerMetricsGroup.java ########## @@ -0,0 +1,204 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Frequencies; +import org.apache.kafka.connect.util.ConnectorTaskId; + +import java.util.Map; + +class WorkerMetricsGroup { + private final ConnectMetrics.MetricGroup metricGroup; + private final Sensor connectorStartupAttempts; + private final Sensor connectorStartupSuccesses; + private final Sensor connectorStartupFailures; + private final Sensor connectorStartupResults; + private final Sensor taskStartupAttempts; + private final Sensor taskStartupSuccesses; + private final Sensor taskStartupFailures; + private final Sensor taskStartupResults; + + public WorkerMetricsGroup(final Map<String, WorkerConnector> connectors, Map<ConnectorTaskId, WorkerTask> tasks, ConnectMetrics connectMetrics) { + ConnectMetricsRegistry registry = connectMetrics.registry(); + metricGroup = connectMetrics.group(registry.workerGroupName()); + + metricGroup.addValueMetric(registry.connectorCount, now -> (double) connectors.size()); + metricGroup.addValueMetric(registry.taskCount, now -> (double) tasks.size()); + + MetricName connectorFailurePct = metricGroup.metricName(registry.connectorStartupFailurePercentage); + MetricName connectorSuccessPct = metricGroup.metricName(registry.connectorStartupSuccessPercentage); + Frequencies connectorStartupResultFrequencies = Frequencies.forBooleanValues(connectorFailurePct, connectorSuccessPct); + connectorStartupResults = metricGroup.sensor("connector-startup-results"); + connectorStartupResults.add(connectorStartupResultFrequencies); + + connectorStartupAttempts = metricGroup.sensor("connector-startup-attempts"); + connectorStartupAttempts.add(metricGroup.metricName(registry.connectorStartupAttemptsTotal), new CumulativeSum()); + + connectorStartupSuccesses = metricGroup.sensor("connector-startup-successes"); + connectorStartupSuccesses.add(metricGroup.metricName(registry.connectorStartupSuccessTotal), new CumulativeSum()); + + connectorStartupFailures = metricGroup.sensor("connector-startup-failures"); + connectorStartupFailures.add(metricGroup.metricName(registry.connectorStartupFailureTotal), new CumulativeSum()); + + MetricName taskFailurePct = metricGroup.metricName(registry.taskStartupFailurePercentage); + MetricName taskSuccessPct = metricGroup.metricName(registry.taskStartupSuccessPercentage); + Frequencies taskStartupResultFrequencies = Frequencies.forBooleanValues(taskFailurePct, taskSuccessPct); + taskStartupResults = metricGroup.sensor("task-startup-results"); + taskStartupResults.add(taskStartupResultFrequencies); + + taskStartupAttempts = metricGroup.sensor("task-startup-attempts"); + taskStartupAttempts.add(metricGroup.metricName(registry.taskStartupAttemptsTotal), new CumulativeSum()); + + taskStartupSuccesses = metricGroup.sensor("task-startup-successes"); + taskStartupSuccesses.add(metricGroup.metricName(registry.taskStartupSuccessTotal), new CumulativeSum()); + + taskStartupFailures = metricGroup.sensor("task-startup-failures"); + taskStartupFailures.add(metricGroup.metricName(registry.taskStartupFailureTotal), new CumulativeSum()); + } + + void close() { + metricGroup.close(); + } + + void recordConnectorStartupFailure() { + connectorStartupAttempts.record(1.0); + connectorStartupFailures.record(1.0); + connectorStartupResults.record(0.0); + } + + void recordConnectorStartupSuccess() { + connectorStartupAttempts.record(1.0); + connectorStartupSuccesses.record(1.0); + connectorStartupResults.record(1.0); + } + + void recordTaskFailure() { + taskStartupAttempts.record(1.0); + taskStartupFailures.record(1.0); + taskStartupResults.record(0.0); + } + + void recordTaskSuccess() { + taskStartupAttempts.record(1.0); + taskStartupSuccesses.record(1.0); + taskStartupResults.record(1.0); + } + + protected ConnectMetrics.MetricGroup metricGroup() { + return metricGroup; + } + + ConnectorStatus.Listener wrapStatusListener(ConnectorStatus.Listener delegateListener) { + return new ConnectorStatusListener(delegateListener); + } + + TaskStatus.Listener wrapStatusListener(TaskStatus.Listener delegateListener) { + return new TaskStatusListener(delegateListener); + } + + class ConnectorStatusListener implements ConnectorStatus.Listener { + private final ConnectorStatus.Listener delegateListener; + private boolean startupSucceeded = false; + + ConnectorStatusListener(ConnectorStatus.Listener delegateListener) { + this.delegateListener = delegateListener; + } + + @Override + public void onShutdown(final String connector) { + delegateListener.onShutdown(connector); + } + + @Override + public void onFailure(final String connector, final Throwable cause) { + if (!startupSucceeded) { + recordConnectorStartupFailure(); + } + delegateListener.onFailure(connector, cause); + } + + @Override + public void onPause(final String connector) { + delegateListener.onPause(connector); + } + + @Override + public void onResume(final String connector) { + delegateListener.onResume(connector); + } + + @Override + public void onStartup(final String connector) { + delegateListener.onStartup(connector); + startupSucceeded = true; + recordConnectorStartupSuccess(); + } + + @Override + public void onDeletion(final String connector) { + delegateListener.onDeletion(connector); + } + } + + class TaskStatusListener implements TaskStatus.Listener { + private final TaskStatus.Listener delegatedListener; + private boolean startupSucceeded = false; Review comment: This is going to be modified and accessed on potentially different threads, right? If so, we should add the `volatile` modifier here. ########## File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerMetricsGroup.java ########## @@ -0,0 +1,204 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.CumulativeSum; +import org.apache.kafka.common.metrics.stats.Frequencies; +import org.apache.kafka.connect.util.ConnectorTaskId; + +import java.util.Map; + +class WorkerMetricsGroup { + private final ConnectMetrics.MetricGroup metricGroup; + private final Sensor connectorStartupAttempts; + private final Sensor connectorStartupSuccesses; + private final Sensor connectorStartupFailures; + private final Sensor connectorStartupResults; + private final Sensor taskStartupAttempts; + private final Sensor taskStartupSuccesses; + private final Sensor taskStartupFailures; + private final Sensor taskStartupResults; + + public WorkerMetricsGroup(final Map<String, WorkerConnector> connectors, Map<ConnectorTaskId, WorkerTask> tasks, ConnectMetrics connectMetrics) { + ConnectMetricsRegistry registry = connectMetrics.registry(); + metricGroup = connectMetrics.group(registry.workerGroupName()); + + metricGroup.addValueMetric(registry.connectorCount, now -> (double) connectors.size()); + metricGroup.addValueMetric(registry.taskCount, now -> (double) tasks.size()); + + MetricName connectorFailurePct = metricGroup.metricName(registry.connectorStartupFailurePercentage); + MetricName connectorSuccessPct = metricGroup.metricName(registry.connectorStartupSuccessPercentage); + Frequencies connectorStartupResultFrequencies = Frequencies.forBooleanValues(connectorFailurePct, connectorSuccessPct); + connectorStartupResults = metricGroup.sensor("connector-startup-results"); + connectorStartupResults.add(connectorStartupResultFrequencies); + + connectorStartupAttempts = metricGroup.sensor("connector-startup-attempts"); + connectorStartupAttempts.add(metricGroup.metricName(registry.connectorStartupAttemptsTotal), new CumulativeSum()); + + connectorStartupSuccesses = metricGroup.sensor("connector-startup-successes"); + connectorStartupSuccesses.add(metricGroup.metricName(registry.connectorStartupSuccessTotal), new CumulativeSum()); + + connectorStartupFailures = metricGroup.sensor("connector-startup-failures"); + connectorStartupFailures.add(metricGroup.metricName(registry.connectorStartupFailureTotal), new CumulativeSum()); + + MetricName taskFailurePct = metricGroup.metricName(registry.taskStartupFailurePercentage); + MetricName taskSuccessPct = metricGroup.metricName(registry.taskStartupSuccessPercentage); + Frequencies taskStartupResultFrequencies = Frequencies.forBooleanValues(taskFailurePct, taskSuccessPct); + taskStartupResults = metricGroup.sensor("task-startup-results"); + taskStartupResults.add(taskStartupResultFrequencies); + + taskStartupAttempts = metricGroup.sensor("task-startup-attempts"); + taskStartupAttempts.add(metricGroup.metricName(registry.taskStartupAttemptsTotal), new CumulativeSum()); + + taskStartupSuccesses = metricGroup.sensor("task-startup-successes"); + taskStartupSuccesses.add(metricGroup.metricName(registry.taskStartupSuccessTotal), new CumulativeSum()); + + taskStartupFailures = metricGroup.sensor("task-startup-failures"); + taskStartupFailures.add(metricGroup.metricName(registry.taskStartupFailureTotal), new CumulativeSum()); + } + + void close() { + metricGroup.close(); + } + + void recordConnectorStartupFailure() { + connectorStartupAttempts.record(1.0); + connectorStartupFailures.record(1.0); + connectorStartupResults.record(0.0); + } + + void recordConnectorStartupSuccess() { + connectorStartupAttempts.record(1.0); + connectorStartupSuccesses.record(1.0); + connectorStartupResults.record(1.0); + } + + void recordTaskFailure() { + taskStartupAttempts.record(1.0); + taskStartupFailures.record(1.0); + taskStartupResults.record(0.0); + } + + void recordTaskSuccess() { + taskStartupAttempts.record(1.0); + taskStartupSuccesses.record(1.0); + taskStartupResults.record(1.0); + } + + protected ConnectMetrics.MetricGroup metricGroup() { + return metricGroup; + } + + ConnectorStatus.Listener wrapStatusListener(ConnectorStatus.Listener delegateListener) { + return new ConnectorStatusListener(delegateListener); + } + + TaskStatus.Listener wrapStatusListener(TaskStatus.Listener delegateListener) { + return new TaskStatusListener(delegateListener); + } + + class ConnectorStatusListener implements ConnectorStatus.Listener { + private final ConnectorStatus.Listener delegateListener; + private boolean startupSucceeded = false; Review comment: This is going to be modified and accessed on potentially different threads, right? If so, we should add the `volatile` modifier here. ########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerMetricsGroupTest.java ########## @@ -0,0 +1,164 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.easymock.EasyMock.eq; +import static org.powermock.api.easymock.PowerMock.createStrictMock; +import static org.powermock.api.easymock.PowerMock.expectLastCall; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Sensor.class}) +public class WorkerMetricsGroupTest { + private final String connector = "org.FakeConnector"; + private final ConnectorTaskId task = new ConnectorTaskId(connector, 0); + private final RuntimeException exception = new RuntimeException(); + + @Test + public void testConnectorStartupRecordedMetrics() { + final WorkerMetricsGroup mockWorkerMetricsGroup = createStrictMock(WorkerMetricsGroup.class); + final ConnectorStatus.Listener delegate = createStrictMock(ConnectorStatus.Listener.class); Review comment: Nit: can probably just use the `@Mock` annotation and make these instance instead of local variables so that we don't have to repeat this code at the beginning of each test. ########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerMetricsGroupTest.java ########## @@ -0,0 +1,164 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.easymock.EasyMock.eq; +import static org.powermock.api.easymock.PowerMock.createStrictMock; +import static org.powermock.api.easymock.PowerMock.expectLastCall; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Sensor.class}) +public class WorkerMetricsGroupTest { + private final String connector = "org.FakeConnector"; + private final ConnectorTaskId task = new ConnectorTaskId(connector, 0); + private final RuntimeException exception = new RuntimeException(); + + @Test + public void testConnectorStartupRecordedMetrics() { + final WorkerMetricsGroup mockWorkerMetricsGroup = createStrictMock(WorkerMetricsGroup.class); + final ConnectorStatus.Listener delegate = createStrictMock(ConnectorStatus.Listener.class); + final WorkerMetricsGroup.ConnectorStatusListener connectorListener = mockWorkerMetricsGroup.new ConnectorStatusListener(delegate); + + delegate.onStartup(connector); Review comment: Nit: looks like similar calls use `eq(connector)` instead of `connector`. I think they both work but we should stick to one or the other. ########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerMetricsGroupTest.java ########## @@ -0,0 +1,164 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.easymock.EasyMock.eq; +import static org.powermock.api.easymock.PowerMock.createStrictMock; +import static org.powermock.api.easymock.PowerMock.expectLastCall; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Sensor.class}) +public class WorkerMetricsGroupTest { + private final String connector = "org.FakeConnector"; + private final ConnectorTaskId task = new ConnectorTaskId(connector, 0); + private final RuntimeException exception = new RuntimeException(); + + @Test + public void testConnectorStartupRecordedMetrics() { + final WorkerMetricsGroup mockWorkerMetricsGroup = createStrictMock(WorkerMetricsGroup.class); + final ConnectorStatus.Listener delegate = createStrictMock(ConnectorStatus.Listener.class); Review comment: Ahh, I see--we construct a connector status listener for some and a task status listener for others. Honestly, I think it's probably fine if we just make both available as instance variables and `@Mock` them. ########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerMetricsGroupTest.java ########## @@ -0,0 +1,164 @@ +/* + * 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.kafka.connect.runtime; + +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.easymock.EasyMock.eq; +import static org.powermock.api.easymock.PowerMock.createStrictMock; +import static org.powermock.api.easymock.PowerMock.expectLastCall; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Sensor.class}) +public class WorkerMetricsGroupTest { + private final String connector = "org.FakeConnector"; + private final ConnectorTaskId task = new ConnectorTaskId(connector, 0); + private final RuntimeException exception = new RuntimeException(); + + @Test + public void testConnectorStartupRecordedMetrics() { + final WorkerMetricsGroup mockWorkerMetricsGroup = createStrictMock(WorkerMetricsGroup.class); + final ConnectorStatus.Listener delegate = createStrictMock(ConnectorStatus.Listener.class); + final WorkerMetricsGroup.ConnectorStatusListener connectorListener = mockWorkerMetricsGroup.new ConnectorStatusListener(delegate); + + delegate.onStartup(connector); + expectLastCall(); + + mockWorkerMetricsGroup.recordConnectorStartupSuccess(); + expectLastCall(); + + PowerMock.replayAll(); + + connectorListener.onStartup(connector); + + PowerMock.verifyAll(); + } + + @Test + public void testConnectorFailureAfterStartupRecordedMetrics() { + final WorkerMetricsGroup mockWorkerMetricsGroup = createStrictMock(WorkerMetricsGroup.class); + final ConnectorStatus.Listener delegate = createStrictMock(ConnectorStatus.Listener.class); + final WorkerMetricsGroup.ConnectorStatusListener connectorListener = mockWorkerMetricsGroup.new ConnectorStatusListener(delegate); + + delegate.onStartup(eq(connector)); + expectLastCall(); + + delegate.onFailure(eq(connector), eq(exception)); + expectLastCall(); + + mockWorkerMetricsGroup.recordConnectorStartupSuccess(); Review comment: Nit: I think it might make more sense if the expectations are set in chronological order instead of grouping by which mocked instance is having expectations set. So in this case, this line would be moved after the expectation for `delegate::onStartup` and before the one for `delegate::onFailure`. But not a big deal, if you think this is more readable feel free to leave as-is. ########## File path: connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java ########## @@ -336,14 +336,11 @@ public void testStartConnectorFailure() throws Exception { assertEquals(exception, e.getCause()); } - assertStartupStatistics(worker, 1, 1, 0, 0); Review comment: It's unfortunate that we're losing test coverage here, especially since it makes issues like the one that necessitates this PR more likely as we can't prevent regressions. Is there a way we can keep some of this testing logic, either through modifying the `WorkerTest` or by relocating it to the `WorkerMetricsGroupTest`? ---------------------------------------------------------------- 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: us...@infra.apache.org