You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/05/15 14:56:14 UTC

[GitHub] [flink] KurtYoung commented on a change in pull request #12069: [FLINK-17734][streaming] Add specialized collecting sink function

KurtYoung commented on a change in pull request #12069:
URL: https://github.com/apache/flink/pull/12069#discussion_r425848753



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkOperatorCoordinator.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.flink.streaming.api.operators.collect;
+
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequest;
+import org.apache.flink.runtime.operators.coordination.CoordinationRequestHandler;
+import org.apache.flink.runtime.operators.coordination.CoordinationResponse;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.util.ExecutorThreadFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+/**
+ * {@link OperatorCoordinator} for {@link CollectSinkFunction}.
+ *
+ * <p>This coordinator only forwards requests and responses from clients and sinks
+ * and it does not store any results in itself.
+ */
+public class CollectSinkOperatorCoordinator implements OperatorCoordinator, CoordinationRequestHandler {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkOperatorCoordinator.class);
+
+	private InetSocketAddress address;
+	private Socket socket;
+	private DataInputViewStreamWrapper inStream;
+	private DataOutputViewStreamWrapper outStream;
+
+	private ExecutorService executorService;
+
+	@Override
+	public void start() throws Exception {
+		this.executorService =
+			Executors.newSingleThreadExecutor(
+				new ExecutorThreadFactory(
+					"collect-sink-operator-coordinator-executor-thread-pool"));
+	}
+
+	@Override
+	public void close() throws Exception {
+		this.executorService.shutdown();
+		closeCurrentConnection();
+	}
+
+	@Override
+	public void handleEventFromOperator(int subtask, OperatorEvent event) throws Exception {
+		Preconditions.checkArgument(
+			event instanceof CollectSinkAddressEvent, "Operator event must be a CollectSinkAddressEvent");
+		address = ((CollectSinkAddressEvent) event).getAddress();
+		LOG.info("Received sink socket server address: " + address);
+
+		// this event is sent when the sink function starts, so we remove the old socket if it is present
+		closeCurrentConnection();
+	}
+
+	@Override
+	public CompletableFuture<CoordinationResponse> handleCoordinationRequest(CoordinationRequest request) {
+		Preconditions.checkArgument(
+			request instanceof CollectCoordinationRequest,
+			"Coordination request must be a CollectCoordinationRequest");
+
+		CompletableFuture<CoordinationResponse> responseFuture = new CompletableFuture<>();
+		executorService.submit(() -> handleRequestImpl((CollectCoordinationRequest) request, responseFuture));
+		return responseFuture;
+	}
+
+	private void handleRequestImpl(
+			CollectCoordinationRequest request,
+			CompletableFuture<CoordinationResponse> responseFuture) {
+		// we back up the address object here to avoid concurrent modifying from `handleEventFromOperator`.
+		// it's ok if this address becomes invalid in the following code,
+		// if this happens, the coordinator will just return an empty result
+		InetSocketAddress address = this.address;
+
+		if (address == null) {
+			completeWithEmptyResponse(request, responseFuture);
+			return;
+		}
+
+		try {
+			if (socket == null) {
+				socket = new Socket(address.getAddress(), address.getPort());
+				socket.setKeepAlive(true);
+				socket.setTcpNoDelay(true);
+				inStream = new DataInputViewStreamWrapper(socket.getInputStream());
+				outStream = new DataOutputViewStreamWrapper(socket.getOutputStream());
+				LOG.info("Sink connection established");
+			}
+
+			// send version and offset to sink server
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Forwarding request to sink socket server");
+			}
+			request.serialize(outStream);
+
+			// fetch back serialized results
+			if (LOG.isDebugEnabled()) {
+				LOG.debug("Fetching serialized result from sink socket server");
+			}
+			responseFuture.complete(new CollectCoordinationResponse(inStream));
+		} catch (IOException e) {
+			// request failed, close current connection and send back empty results
+			// we catch every exception here because socket might suddenly becomes null if the sink fails
+			// and we do not want the coordinator to fail
+			closeCurrentConnection();
+			completeWithEmptyResponse(request, responseFuture);
+		}
+	}
+
+	private void completeWithEmptyResponse(
+			CollectCoordinationRequest request,
+			CompletableFuture<CoordinationResponse> future) {
+		try {
+			future.complete(new CollectCoordinationResponse<>(
+				request.getVersion(),
+				// this lastCheckpointedOffset is OK
+				// because client will only expose results to the users when the checkpointed offset increases
+				0,

Review comment:
       Better to return -1 here? Because 0 is also a valid `last checkpointed offset`

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java
##########
@@ -0,0 +1,425 @@
+/*
+ * 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.flink.streaming.api.operators.collect;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.accumulators.SerializedListAccumulator;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A sink function that collects query results and sends them back to the client.
+ *
+ * <p>This sink works by limiting the number of results buffered in it (can be configured) so
+ * that when the buffer is full, it back-pressures the job until the client consumes some results.
+ *
+ * <p>NOTE: When using this sink, make sure that its parallelism is 1, and make sure that it is used
+ * in a {@link StreamTask}.
+ *
+ * <h2>Communication Protocol Explanation</h2>
+ *
+ * <p>We maintain the following variables in this communication protocol
+ * <ol>
+ *     <li><strong>version</strong>: This variable will be set to a random value when the sink opens.
+ *         Client discovers that the sink has restarted if this variable is different.</li>
+ *     <li><strong>offset</strong>: This indicates that client has successfully received the results
+ *         before this offset. Sink can safely throw these results away.</li>
+ *     <li><strong>lastCheckpointedOffset</strong>:
+ *         This is the value of <code>offset</code> when the checkpoint happens. This value will be
+ *         restored from the checkpoint and set back to <code>offset</code> when the sink restarts.</li>
+ * </ol>
+ *
+ * <p>Client will put <code>version</code> and <code>offset</code> into the request, indicating that
+ * it thinks what the current version is and it has received this much results.
+ *
+ * <p>Sink will check the validity of the request. If <code>version</code> mismatches or <code>offset</code>
+ * is smaller than expected, sink will send back the current <code>version</code> and
+ * <code>lastCheckpointedOffset</code> with an empty result list to indicate an invalid request.
+ *
+ * <p>If the request is valid, sink prepares some results starting from <code>offset</code> and sends them
+ * back to the client with <code>lastCheckpointedOffset</code>.
+ *
+ * <p>For client who wants exactly-once semantics, when receiving the response, the client will check for
+ * the following conditions:
+ * <ol>
+ *     <li>If the version mismatches, client knows that sink has restarted. It will throw away all uncheckpointed
+ *         results after <code>lastCheckpointedOffset</code>.</li>
+ *     <li>Otherwise the version matches. If <code>lastCheckpointedOffset</code> increases, client knows that
+ *         a checkpoint happens. It can now move all results before this offset to a user-visible buffer. If
+ *         the response also contains new results, client will now move these new results into uncheckpointed
+ *         buffer.</li>
+ * </ol>
+ *
+ * <p>Note that
+ * <ol>
+ *     <li>user can only see results before a <code>lastCheckpointedOffset</code>, and</li>
+ *     <li>client will go back to the latest <code>lastCheckpointedOffset</code> when sink restarts,</li>
+ * </ol>
+ * client will never throw away results in user-visible buffer.
+ * So this communication protocol achieves exactly-once semantics.
+ *
+ * @param <IN> type of results to be written into the sink.
+ */
+@Internal
+public class CollectSinkFunction<IN> extends RichSinkFunction<IN> implements CheckpointedFunction, CheckpointListener {
+
+	private static final Logger LOG = LoggerFactory.getLogger(CollectSinkFunction.class);
+
+	private final TypeSerializer<IN> serializer;
+	private final int maxResultsPerBatch;
+	private final int maxResultsBuffered;
+	private final String finalResultListAccumulatorName;
+	private final String finalResultOffsetAccumulatorName;
+
+	private transient OperatorEventGateway eventGateway;
+
+	private transient LinkedList<IN> bufferedResults;
+	private transient ReentrantLock bufferedResultsLock;
+	private transient Condition bufferNotFullCondition;
+
+	// this version indicates whether the sink has restarted or not
+	private transient String version;
+	// this offset acts as an acknowledgement,
+	// results before this offset can be safely thrown away
+	private transient long offset;
+	private transient long lastCheckpointedOffset;
+
+	private transient ServerThread serverThread;
+
+	private transient ListState<IN> bufferedResultsState;
+	private transient ListState<Long> offsetState;
+	private transient SortedMap<Long, Long> uncompletedCheckpointMap;
+
+	public CollectSinkFunction(
+			TypeSerializer<IN> serializer,
+			int maxResultsPerBatch,
+			String finalResultListAccumulatorName,
+			String finalResultOffsetAccumulatorName) {
+		this.serializer = serializer;
+		this.maxResultsPerBatch = maxResultsPerBatch;
+		this.maxResultsBuffered = maxResultsPerBatch * 2;
+		this.finalResultListAccumulatorName = finalResultListAccumulatorName;
+		this.finalResultOffsetAccumulatorName = finalResultOffsetAccumulatorName;
+	}
+
+	private void initBuffer() {
+		if (bufferedResults != null) {
+			return;
+		}
+
+		bufferedResults = new LinkedList<>();
+		bufferedResultsLock = new ReentrantLock();
+		bufferNotFullCondition = bufferedResultsLock.newCondition();
+
+		offset = 0;
+		lastCheckpointedOffset = offset;
+	}
+
+	@Override
+	public void initializeState(FunctionInitializationContext context) throws Exception {
+		initBuffer();
+
+		bufferedResultsState =
+			context.getOperatorStateStore().getListState(
+				new ListStateDescriptor<>("bufferedResultsState", serializer));
+		bufferedResults.clear();
+		for (IN result : bufferedResultsState.get()) {
+			bufferedResults.add(result);
+		}
+
+		offsetState = context.getOperatorStateStore().getListState(
+			new ListStateDescriptor<>("offsetState", Long.class));
+		offset = 0;
+		// there must be only 1 element in this state when restoring
+		for (long value : offsetState.get()) {
+			offset = value;
+		}
+		lastCheckpointedOffset = offset;
+
+		uncompletedCheckpointMap = new TreeMap<>();
+	}
+
+	@Override
+	public void snapshotState(FunctionSnapshotContext context) throws Exception {
+		bufferedResultsLock.lock();
+		try {
+			bufferedResultsState.clear();
+			bufferedResultsState.addAll(bufferedResults);
+
+			offsetState.clear();
+			offsetState.add(offset);
+
+			uncompletedCheckpointMap.put(context.getCheckpointId(), offset);
+		} finally {
+			bufferedResultsLock.unlock();
+		}
+	}
+
+	@Override
+	public void open(Configuration parameters) throws Exception {
+		Preconditions.checkState(
+			getRuntimeContext().getNumberOfParallelSubtasks() == 1,
+			"The parallelism of CollectSinkFunction must be 1");
+
+		initBuffer();
+
+		// generate a random uuid when the sink is opened
+		// so that the client can know if the sink has been restarted
+		version = UUID.randomUUID().toString();
+
+		serverThread = new ServerThread();
+		serverThread.start();
+
+		// sending socket server address to coordinator
+		Preconditions.checkNotNull(eventGateway, "Operator event gateway hasn't been set");
+		InetSocketAddress address = serverThread.getServerSocketAddress();
+		LOG.info("Collect sink server established, address = " + address);
+
+		CollectSinkAddressEvent addressEvent = new CollectSinkAddressEvent(address);
+		eventGateway.sendEventToCoordinator(addressEvent);
+	}
+
+	@Override
+	public void invoke(IN value, Context context) throws Exception {
+		bufferedResultsLock.lock();
+		try {
+			if (bufferedResults.size() >= maxResultsBuffered) {
+				bufferNotFullCondition.await();
+			}
+			bufferedResults.add(value);
+		} finally {
+			bufferedResultsLock.unlock();
+		}
+	}
+
+	@Override
+	public void close() throws Exception {
+		serverThread.close();
+	}
+
+	public void accumulateFinalResults() throws Exception {
+		bufferedResultsLock.lock();
+		try {
+			// put results not consumed by the client into the accumulator
+			// so that we do not block the closing procedure while not throwing results away
+			SerializedListAccumulator<IN> listAccumulator = new SerializedListAccumulator<>();
+			for (IN result : bufferedResults) {
+				listAccumulator.add(result, serializer);
+			}
+			LongCounter offsetAccumulator = new LongCounter(offset);
+			getRuntimeContext().addAccumulator(finalResultListAccumulatorName, listAccumulator);
+			getRuntimeContext().addAccumulator(finalResultOffsetAccumulatorName, offsetAccumulator);
+		} finally {
+			bufferedResultsLock.unlock();
+		}
+	}
+
+	@Override
+	public void notifyCheckpointComplete(long checkpointId) {
+		lastCheckpointedOffset = uncompletedCheckpointMap.get(checkpointId);
+		uncompletedCheckpointMap.headMap(checkpointId + 1).clear();
+	}
+
+	public void setOperatorEventGateway(OperatorEventGateway eventGateway) {
+		this.eventGateway = eventGateway;
+	}
+
+	/**
+	 * The thread that runs the socket server.
+	 */
+	private class ServerThread extends Thread {
+
+		private final ServerSocket serverSocket;
+
+		private boolean running;
+
+		private Socket connection;
+		private DataInputViewStreamWrapper inStream;
+		private DataOutputViewStreamWrapper outStream;
+
+		private ServerThread() throws Exception {
+			this.serverSocket = new ServerSocket(0, 0, getBindAddress());
+			this.running = true;
+		}
+
+		@Override
+		public void run() {
+			while (running) {
+				try {
+					if (connection == null) {
+						// waiting for coordinator to connect
+						connection = serverSocket.accept();
+						inStream = new DataInputViewStreamWrapper(this.connection.getInputStream());
+						outStream = new DataOutputViewStreamWrapper(this.connection.getOutputStream());
+						LOG.info("Coordinator connection received");
+					}
+
+					CollectCoordinationRequest request = new CollectCoordinationRequest(inStream);
+					String requestVersion = request.getVersion();
+					// client acknowledges that it has successfully received results before this offset,
+					// we can safely throw away results before this offset
+					long requestOffset = request.getOffset();
+					if (LOG.isDebugEnabled()) {
+						LOG.debug(
+							"Request received, version = " + requestVersion + ", offset = " + requestOffset);
+						LOG.debug(
+							"Expecting version = " + version + ", offset = " + offset);
+					}
+
+					if (!version.equals(requestVersion) || requestOffset < offset) {
+						// invalid request
+						LOG.warn("Invalid request. Received version = " + requestVersion +
+							", offset = " + requestOffset + ", while expected version = "
+							+ version + ", offset = " + offset);
+						sendBackResults(Collections.emptyList());
+						continue;
+					}
+
+					// valid request, sending out results
+					List<IN> results;
+					bufferedResultsLock.lock();
+					try {
+						int oldSize = bufferedResults.size();
+						int ackedNum = Math.min((int) (requestOffset - offset), oldSize);
+						int nextBatchSize = Math.min(ackedNum + maxResultsPerBatch, oldSize) - ackedNum;
+
+						if (LOG.isDebugEnabled()) {
+							LOG.debug("Preparing " + nextBatchSize + " results");
+						}
+
+						// drop acked results
+						for (int i = 0; i < ackedNum; i++) {
+							bufferedResults.removeFirst();
+							offset++;
+						}
+
+						// prepare next result batch
+						results = new ArrayList<>(bufferedResults.subList(0, nextBatchSize));

Review comment:
       `results = bufferedResults.subList(0, nextBatchSize);` ?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java
##########
@@ -0,0 +1,425 @@
+/*
+ * 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.flink.streaming.api.operators.collect;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.accumulators.LongCounter;
+import org.apache.flink.api.common.accumulators.SerializedListAccumulator;
+import org.apache.flink.api.common.functions.RuntimeContext;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.UUID;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A sink function that collects query results and sends them back to the client.
+ *
+ * <p>This sink works by limiting the number of results buffered in it (can be configured) so
+ * that when the buffer is full, it back-pressures the job until the client consumes some results.
+ *
+ * <p>NOTE: When using this sink, make sure that its parallelism is 1, and make sure that it is used
+ * in a {@link StreamTask}.
+ *
+ * <h2>Communication Protocol Explanation</h2>
+ *
+ * <p>We maintain the following variables in this communication protocol
+ * <ol>
+ *     <li><strong>version</strong>: This variable will be set to a random value when the sink opens.
+ *         Client discovers that the sink has restarted if this variable is different.</li>
+ *     <li><strong>offset</strong>: This indicates that client has successfully received the results
+ *         before this offset. Sink can safely throw these results away.</li>
+ *     <li><strong>lastCheckpointedOffset</strong>:
+ *         This is the value of <code>offset</code> when the checkpoint happens. This value will be
+ *         restored from the checkpoint and set back to <code>offset</code> when the sink restarts.</li>
+ * </ol>
+ *
+ * <p>Client will put <code>version</code> and <code>offset</code> into the request, indicating that
+ * it thinks what the current version is and it has received this much results.
+ *
+ * <p>Sink will check the validity of the request. If <code>version</code> mismatches or <code>offset</code>
+ * is smaller than expected, sink will send back the current <code>version</code> and
+ * <code>lastCheckpointedOffset</code> with an empty result list to indicate an invalid request.

Review comment:
       If client sends a valid offset, but there is currently no new data, we will also send back an empty list. So empty result list not always indicating an invalid request.




----------------------------------------------------------------
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:
users@infra.apache.org