abhishekbafna commented on code in PR #17521:
URL: https://github.com/apache/pinot/pull/17521#discussion_r2895519430
##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -372,6 +388,9 @@ public CopyTableResponse copyTable(
response.setTableConfig(realtimeTableConfig);
response.setWatermarkInductionResult(watermarkInductionResult);
}
+ String jobID = UUID.randomUUID().toString();
+ _tableReplicator.replicateTable(jobID,
realtimeTableConfig.getTableName(), copyTablePayload,
Review Comment:
How do we handle controller restart/crash events? The running job would be
killed and would not start again. How do users resumes the replication back?
This is specially important when working with large tables.
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/replication/ZkBasedTableReplicationObserver.java:
##########
@@ -0,0 +1,97 @@
+/**
+ * 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.pinot.controller.helix.core.replication;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.util.Map;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.WatermarkInductionResult;
+import org.apache.pinot.controller.helix.core.controllerjob.ControllerJobTypes;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Observes the table replication progress and updates the status in Zookeeper.
+ */
+public class ZkBasedTableReplicationObserver implements
TableReplicationObserver {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(ZkBasedTableReplicationObserver.class);
+
+ private final PinotHelixResourceManager _pinotHelixResourceManager;
+ private final String _jobId;
+ private final String _tableNameWithType;
+ private final TableReplicationProgressStats _progressStats;
+ private final WatermarkInductionResult _res;
+
+ public ZkBasedTableReplicationObserver(String jobId, String
tableNameWithType, WatermarkInductionResult res,
+ PinotHelixResourceManager pinotHelixResourceManager) {
+ _jobId = jobId;
+ _tableNameWithType = tableNameWithType;
+ _res = res;
+ _pinotHelixResourceManager = pinotHelixResourceManager;
+ _progressStats = new
TableReplicationProgressStats(res.getHistoricalSegments().size());
+ }
+
+ @Override
+ public void onTrigger(Trigger trigger, String segmentName) {
+ switch (trigger) {
+ // Table
+ case START_TRIGGER:
+ // in case of zero segments to be copied, track stats in ZK
+ trackStatsInZk();
+ break;
+ case SEGMENT_REPLICATE_COMPLETED_TRIGGER:
+ // Update progress stats and track in ZK every 100 segments
+ int remaining = _progressStats.updateSegmentStatus(segmentName,
+ TableReplicationProgressStats.SegmentStatus.COMPLETED);
+ if (remaining % 100 == 0) {
+ trackStatsInZk();
+ }
+ break;
+ case SEGMENT_REPLICATE_ERRORED_TRIGGER:
+ // Update progress stats and track in ZK immediately on error
+ _progressStats.updateSegmentStatus(segmentName,
TableReplicationProgressStats.SegmentStatus.ERROR);
+ trackStatsInZk();
+ break;
+ default:
+ }
+ }
+
+ private void trackStatsInZk() {
+ LOGGER.info("[copyTable] Tracking replication stats in ZK for job: {}",
_jobId);
+ try {
+ Map<String, String> jobMetadata = _pinotHelixResourceManager
+ .commonTableReplicationJobMetadata(_tableNameWithType, _jobId,
System.currentTimeMillis(), _res);
+ String progress = JsonUtils.objectToString(_progressStats);
+ jobMetadata.put(CommonConstants.ControllerJob.REPLICATION_PROGRESS,
progress);
+ int remaining = JsonUtils.stringToObject(progress,
JsonNode.class).get("remainingSegments").asInt();
+ if (remaining == 0) {
+ jobMetadata.put(CommonConstants.ControllerJob.REPLICATION_JOB_STATUS,
"COMPLETED");
+ } else {
+ jobMetadata.put(CommonConstants.ControllerJob.REPLICATION_JOB_STATUS,
"IN_PROGRESS");
+ }
+ _pinotHelixResourceManager.addControllerJobToZK(_jobId, jobMetadata,
ControllerJobTypes.TABLE_REPLICATION);
Review Comment:
How do we handle zk write failure?
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/replication/RealtimeSegmentCopier.java:
##########
@@ -0,0 +1,177 @@
+/**
+ * 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.pinot.controller.helix.core.replication;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+import org.apache.hc.core5.http.ClassicHttpRequest;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.hc.core5.http.HttpVersion;
+import org.apache.hc.core5.http.io.support.ClassicRequestBuilder;
+import org.apache.pinot.common.exception.HttpErrorStatusException;
+import org.apache.pinot.common.utils.FileUploadDownloadClient;
+import org.apache.pinot.common.utils.SimpleHttpResponse;
+import org.apache.pinot.common.utils.http.HttpClient;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.controller.api.resources.CopyTablePayload;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.retry.RetryPolicies;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Copies a realtime segment from source to destination.
+ */
+public class RealtimeSegmentCopier implements SegmentCopier {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(RealtimeSegmentCopier.class);
+ private static final String SEGMENT_UPLOAD_ENDPOINT_TEMPLATE =
"/segments?tableName=%s";
Review Comment:
unused constant
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/replication/ZkBasedTableReplicationObserver.java:
##########
@@ -0,0 +1,97 @@
+/**
+ * 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.pinot.controller.helix.core.replication;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.util.Map;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.WatermarkInductionResult;
+import org.apache.pinot.controller.helix.core.controllerjob.ControllerJobTypes;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.apache.pinot.spi.utils.JsonUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Observes the table replication progress and updates the status in Zookeeper.
+ */
+public class ZkBasedTableReplicationObserver implements
TableReplicationObserver {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(ZkBasedTableReplicationObserver.class);
+
+ private final PinotHelixResourceManager _pinotHelixResourceManager;
+ private final String _jobId;
+ private final String _tableNameWithType;
+ private final TableReplicationProgressStats _progressStats;
+ private final WatermarkInductionResult _res;
+
+ public ZkBasedTableReplicationObserver(String jobId, String
tableNameWithType, WatermarkInductionResult res,
+ PinotHelixResourceManager pinotHelixResourceManager) {
+ _jobId = jobId;
+ _tableNameWithType = tableNameWithType;
+ _res = res;
+ _pinotHelixResourceManager = pinotHelixResourceManager;
+ _progressStats = new
TableReplicationProgressStats(res.getHistoricalSegments().size());
+ }
+
+ @Override
+ public void onTrigger(Trigger trigger, String segmentName) {
+ switch (trigger) {
+ // Table
+ case START_TRIGGER:
+ // in case of zero segments to be copied, track stats in ZK
+ trackStatsInZk();
+ break;
+ case SEGMENT_REPLICATE_COMPLETED_TRIGGER:
+ // Update progress stats and track in ZK every 100 segments
+ int remaining = _progressStats.updateSegmentStatus(segmentName,
+ TableReplicationProgressStats.SegmentStatus.COMPLETED);
+ if (remaining % 100 == 0) {
+ trackStatsInZk();
+ }
+ break;
+ case SEGMENT_REPLICATE_ERRORED_TRIGGER:
+ // Update progress stats and track in ZK immediately on error
+ _progressStats.updateSegmentStatus(segmentName,
TableReplicationProgressStats.SegmentStatus.ERROR);
+ trackStatsInZk();
+ break;
+ default:
+ }
+ }
+
+ private void trackStatsInZk() {
+ LOGGER.info("[copyTable] Tracking replication stats in ZK for job: {}",
_jobId);
+ try {
+ Map<String, String> jobMetadata = _pinotHelixResourceManager
+ .commonTableReplicationJobMetadata(_tableNameWithType, _jobId,
System.currentTimeMillis(), _res);
+ String progress = JsonUtils.objectToString(_progressStats);
+ jobMetadata.put(CommonConstants.ControllerJob.REPLICATION_PROGRESS,
progress);
+ int remaining = JsonUtils.stringToObject(progress,
JsonNode.class).get("remainingSegments").asInt();
+ if (remaining == 0) {
Review Comment:
Should there be another status when there are `segmentsFailToCopy` is non
empty like `COMPLETED_WITH_ERRORS`? How the user would know if there are any
failures?
##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -415,6 +434,18 @@ static void tweakRealtimeTableConfig(ObjectNode
realtimeTableConfigNode, CopyTab
}
}
+ @GET
+ @Path("/tables/copyStatus/{jobId}")
+ @Authorize(targetType = TargetType.CLUSTER, action =
Actions.Cluster.GET_TABLE_COPY_STATUS)
+ @Produces(MediaType.APPLICATION_JSON)
+ @ApiOperation(value = "Get status for a submitted table replication job",
+ notes = "Get status for a submitted table replication job")
+ public JsonNode getForceCommitJobStatus(
+ @ApiParam(value = "job id", required = true) @PathParam("jobId") String
id) {
+ return JsonUtils.objectToJsonNode(
+ _pinotHelixResourceManager.getControllerJobZKMetadata(id,
ControllerJobTypes.TABLE_REPLICATION));
Review Comment:
The `getControllerJobZKMetadata` return null which would return empty
response with 200 status. We should handle that appropriately with relevant
status code.
##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/PinotTableRestletResource.java:
##########
@@ -415,6 +434,18 @@ static void tweakRealtimeTableConfig(ObjectNode
realtimeTableConfigNode, CopyTab
}
}
+ @GET
+ @Path("/tables/copyStatus/{jobId}")
+ @Authorize(targetType = TargetType.CLUSTER, action =
Actions.Cluster.GET_TABLE_COPY_STATUS)
+ @Produces(MediaType.APPLICATION_JSON)
+ @ApiOperation(value = "Get status for a submitted table replication job",
+ notes = "Get status for a submitted table replication job")
+ public JsonNode getForceCommitJobStatus(
Review Comment:
The method name and api does not match.
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/replication/RealtimeSegmentCopier.java:
##########
@@ -0,0 +1,177 @@
+/**
+ * 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.pinot.controller.helix.core.replication;
+
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+import org.apache.hc.core5.http.ClassicHttpRequest;
+import org.apache.hc.core5.http.HttpHeaders;
+import org.apache.hc.core5.http.HttpVersion;
+import org.apache.hc.core5.http.io.support.ClassicRequestBuilder;
+import org.apache.pinot.common.exception.HttpErrorStatusException;
+import org.apache.pinot.common.utils.FileUploadDownloadClient;
+import org.apache.pinot.common.utils.SimpleHttpResponse;
+import org.apache.pinot.common.utils.http.HttpClient;
+import org.apache.pinot.controller.ControllerConf;
+import org.apache.pinot.controller.api.resources.CopyTablePayload;
+import org.apache.pinot.spi.filesystem.PinotFS;
+import org.apache.pinot.spi.filesystem.PinotFSFactory;
+import org.apache.pinot.spi.utils.retry.RetryPolicies;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Copies a realtime segment from source to destination.
+ */
+public class RealtimeSegmentCopier implements SegmentCopier {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(RealtimeSegmentCopier.class);
+ private static final String SEGMENT_UPLOAD_ENDPOINT_TEMPLATE =
"/segments?tableName=%s";
+
+ private final String _destinationDeepStoreUri;
+ private final HttpClient _httpClient;
+
+ public RealtimeSegmentCopier(ControllerConf controllerConf) {
+ this(controllerConf, HttpClient.getInstance());
+ }
+
+ public RealtimeSegmentCopier(ControllerConf controllerConf, HttpClient
httpClient) {
+ _destinationDeepStoreUri = controllerConf.getDataDir();
+ _httpClient = httpClient;
+ }
+
+
+ /**
+ * Copies a segment to the destination cluster.
+ *
+ * This method performs the following steps:
+ * 1. Get the source segment URI from ZK metadata.
+ * 2. Copy the segment from the source deep store to the destination deep
store.
+ * 3. Upload the segment to the destination controller.
+ *
+ * @param tableNameWithType Table name with type suffix
+ * @param segmentName Segment name
+ * @param copyTablePayload Payload for copying a table
+ * @param segmentZKMetadata ZK metadata for the segment
+ */
+ @Override
+ public void copy(String tableNameWithType, String segmentName,
CopyTablePayload copyTablePayload,
+ Map<String, String> segmentZKMetadata) {
+ if (!tableNameWithType.endsWith("_REALTIME")) {
+ throw new IllegalArgumentException("Table name must end with _REALTIME");
+ }
+ String tableName = tableNameWithType.substring(0,
tableNameWithType.lastIndexOf("_REALTIME"));
+ try {
+ // 1. Get the the source segment uri
+ String downloadUrl = segmentZKMetadata.get("segment.download.url");
Review Comment:
Use constant from CommonConstants.Segment class.
##########
pinot-controller/src/main/java/org/apache/pinot/controller/helix/core/replication/TableReplicator.java:
##########
@@ -0,0 +1,131 @@
+/**
+ * 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.pinot.controller.helix.core.replication;
+
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import org.apache.pinot.common.utils.SimpleHttpResponse;
+import org.apache.pinot.common.utils.http.HttpClient;
+import org.apache.pinot.controller.api.resources.CopyTablePayload;
+import org.apache.pinot.controller.helix.core.PinotHelixResourceManager;
+import org.apache.pinot.controller.helix.core.WatermarkInductionResult;
+import org.apache.pinot.spi.utils.builder.ControllerRequestURLBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Replicates a table from a source cluster to a destination cluster.
+ */
+public class TableReplicator {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(TableReplicator.class);
+ private final PinotHelixResourceManager _pinotHelixResourceManager;
+ private final ExecutorService _executorService;
+ private final SegmentCopier _segmentCopier;
+ private final HttpClient _httpClient;
+
+ public TableReplicator(PinotHelixResourceManager pinotHelixResourceManager,
ExecutorService executorService,
+ SegmentCopier segmentCopier) {
+ this(pinotHelixResourceManager, executorService, segmentCopier,
HttpClient.getInstance());
+ }
+
+ public TableReplicator(PinotHelixResourceManager pinotHelixResourceManager,
ExecutorService executorService,
+ SegmentCopier segmentCopier, HttpClient httpClient) {
+ _pinotHelixResourceManager = pinotHelixResourceManager;
+ _executorService = executorService;
+ _segmentCopier = segmentCopier;
+ _httpClient = httpClient;
+ }
+
+ /**
+ * Replicates the table by copying segments from source to destination.
+ *
+ * This method performs the following steps:
+ * 1. Fetch ZK metadata for all segments of the table from the source
cluster.
+ * 2. Register a new controller job in Zookeeper to track the replication
progress.
+ * 3. Initialize a {@link ZkBasedTableReplicationObserver} to update the job
status in Zookeeper.
+ * 4. Submit tasks to the executor service to copy segments in parallel.
+ * 5. Each task copies a segment and triggers the observer to update the
progress.
+ *
+ * @param jobId The job ID.
+ * @param tableNameWithType The table name with type.
+ * @param copyTablePayload The payload containing the source and destination
cluster information.
+ * @param res The watermark induction result.
+ * @throws Exception If an error occurs during replication.
+ */
+ public void replicateTable(String jobId, String tableNameWithType,
CopyTablePayload copyTablePayload,
+ WatermarkInductionResult res)
+ throws Exception {
+ // TODO: throw IllegalStateException if any previous jobs doesn't expire.
+ // TODO: replication job canceling mechanism
+ LOGGER.info("[copyTable] Start replicating table: {} with jobId: {}",
tableNameWithType, jobId);
+ ControllerRequestURLBuilder urlBuilder =
+
ControllerRequestURLBuilder.baseUrl(copyTablePayload.getSourceClusterUri());
+ URI zkMetadataUri = new
URI(urlBuilder.forSegmentZkMetadata(tableNameWithType));
+ SimpleHttpResponse zkMetadataResponse =
HttpClient.wrapAndThrowHttpException(
+ _httpClient.sendGetRequest(zkMetadataUri,
copyTablePayload.getHeaders()));
+ String zkMetadataJson = zkMetadataResponse.getResponse();
+ Map<String, Map<String, String>> zkMetadataMap =
+ new ObjectMapper().readValue(zkMetadataJson, new
TypeReference<Map<String, Map<String, String>>>() {
+ });
+ LOGGER.info("[copyTable] Fetched ZK metadata for {} segments",
zkMetadataMap.size());
+
+ List<String> segments = new ArrayList<>(res.getHistoricalSegments());
+ long submitTS = System.currentTimeMillis();
+
+ if
(!_pinotHelixResourceManager.addNewTableReplicationJob(tableNameWithType,
jobId, submitTS, res)) {
+ throw new Exception("Failed to add segments to replicated table");
+ }
+ ZkBasedTableReplicationObserver observer = new
ZkBasedTableReplicationObserver(jobId, tableNameWithType, res,
+ _pinotHelixResourceManager);
+ observer.onTrigger(TableReplicationObserver.Trigger.START_TRIGGER, null);
+ ConcurrentLinkedQueue<String> q = new ConcurrentLinkedQueue<>(segments);
+ int parallelism = copyTablePayload.getBackfillParallism() != null
+ ? copyTablePayload.getBackfillParallism()
+ : res.getWatermarks().size();
+ for (int i = 0; i < parallelism; i++) {
+ _executorService.submit(() -> {
+ while (true) {
+ String segment = q.poll();
+ if (segment == null) {
+ break;
+ }
+ try {
+ LOGGER.info("[copyTable] Starting to copy segment: {} for table:
{}", segment, tableNameWithType);
+ Map<String, String> segmentZKMetadata = zkMetadataMap.get(segment);
+ if (segmentZKMetadata == null) {
+ throw new RuntimeException("Segment ZK metadata not found for
segment: " + segment);
+ }
+ _segmentCopier.copy(tableNameWithType, segment, copyTablePayload,
segmentZKMetadata);
+
observer.onTrigger(TableReplicationObserver.Trigger.SEGMENT_REPLICATE_COMPLETED_TRIGGER,
segment);
+ } catch (Exception e) {
+ LOGGER.error("Caught exception while replicating table segment",
e);
Review Comment:
Can include table and segment name in the message for debugging?
##########
pinot-controller/src/main/java/org/apache/pinot/controller/api/resources/CopyTablePayload.java:
##########
@@ -40,6 +61,14 @@ public class CopyTablePayload {
* MUST NOT contain the tenant type suffix, i.e. _REALTIME or _OFFLINE.
*/
private String _serverTenant;
+ private Integer _backfillParallism;
Review Comment:
typo in the `Parallism` should be `Parallelism`
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]