snleee commented on code in PR #9295: URL: https://github.com/apache/pinot/pull/9295#discussion_r957981334
########## pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentUploadIntegrationTest.java: ########## @@ -81,10 +89,15 @@ protected List<String> getBloomFilterColumns() { return null; } + @BeforeMethod + public void setUpTest() Review Comment: I think that this is not actually necessary. We can call `TestUtils.ensureDirectoriesExistAndEmpty()` function in `setUp()` and clean up those in `cleanUp()` For 2 different tests, we can use different table names. ########## pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentUploadIntegrationTest.java: ########## @@ -217,10 +358,15 @@ public Boolean apply(@Nullable Void aVoid) { }, 100L, 300_000, "Failed to load " + countStarResult + " documents", true); } + @AfterMethod Review Comment: This is also not necessary if we use different table names. ########## pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentUploadIntegrationTest.java: ########## @@ -188,6 +202,133 @@ public void testUploadAndQuery() testCountStar(numDocs); } + /** + * Runs both SegmentMetadataPushJobRunner and SegmentTarPushJobRunner while enabling consistent data push. + * Checks that segments are properly loaded and segment lineage entry were also in expected states. + */ + @Test + public void testUploadAndQueryWithConsistentPush() + throws Exception { + // Create and upload the schema and table config + Schema schema = createSchema(); + addSchema(schema); + TableConfig offlineTableConfig = createOfflineTableConfigWithConsistentPush(); + addTableConfig(offlineTableConfig); + + List<File> avroFiles = getAllAvroFiles(); + + ClusterIntegrationTestUtils.buildSegmentFromAvro(avroFiles.get(0), offlineTableConfig, schema, "_with_move", + _segmentDir, _tarDir); + + // First test standalone metadata push job runner + BaseSegmentPushJobRunner runner = new SegmentMetadataPushJobRunner(); + SegmentGenerationJobSpec jobSpec = new SegmentGenerationJobSpec(); + PushJobSpec pushJobSpec = new PushJobSpec(); + pushJobSpec.setCopyToDeepStoreForMetadataPush(true); + jobSpec.setPushJobSpec(pushJobSpec); + PinotFSSpec fsSpec = new PinotFSSpec(); + fsSpec.setScheme("file"); + fsSpec.setClassName("org.apache.pinot.spi.filesystem.LocalPinotFS"); + jobSpec.setPinotFSSpecs(Lists.newArrayList(fsSpec)); + jobSpec.setOutputDirURI(_tarDir.getAbsolutePath()); + TableSpec tableSpec = new TableSpec(); + tableSpec.setTableName(DEFAULT_TABLE_NAME); + tableSpec.setTableConfigURI(_controllerRequestURLBuilder.forUpdateTableConfig(DEFAULT_TABLE_NAME)); + jobSpec.setTableSpec(tableSpec); + PinotClusterSpec clusterSpec = new PinotClusterSpec(); + clusterSpec.setControllerURI(_controllerBaseApiUrl); + jobSpec.setPinotClusterSpecs(new PinotClusterSpec[]{clusterSpec}); + + File dataDir = new File(_controllerConfig.getDataDir()); + File dataDirSegments = new File(dataDir, DEFAULT_TABLE_NAME); + + Assert.assertEquals(_tarDir.listFiles().length, 1); + + runner.init(jobSpec); + runner.run(); + + // Segment should be seen in dataDir + Assert.assertTrue(dataDirSegments.exists()); + Assert.assertEquals(dataDirSegments.listFiles().length, 1); + Assert.assertEquals(_tarDir.listFiles().length, 1); + + // test segment loaded + JsonNode segmentsList = getSegmentsList(); + Assert.assertEquals(segmentsList.size(), 1); + String segmentNameWithMove = segmentsList.get(0).asText(); + Assert.assertTrue(segmentNameWithMove.endsWith("_with_move")); + long numDocs = getNumDocs(segmentNameWithMove); + testCountStar(numDocs); + + // Fetch segment lineage entry after running segment metadata push with consistent push enabled. + String segmentLineageResponse = ControllerTest.sendGetRequest( + ControllerRequestURLBuilder.baseUrl(_controllerBaseApiUrl) + .forListAllSegmentLineages(DEFAULT_TABLE_NAME, TableType.OFFLINE.toString())); + // Segment lineage should be in completed state. + Assert.assertTrue(segmentLineageResponse.contains("\"state\":\"COMPLETED\"")); + // SegmentsFrom should be empty as we started with a blank table. + Assert.assertTrue(segmentLineageResponse.contains("\"segmentsFrom\":[]")); + // SegmentsTo should contain uploaded segment. + Assert.assertTrue(segmentLineageResponse.contains("\"segmentsTo\":[\"" + segmentNameWithMove + "\"]")); + + // Clear segment and tar dir + for (File segment : _segmentDir.listFiles()) { + FileUtils.deleteQuietly(segment); + } + for (File tar : _tarDir.listFiles()) { + FileUtils.deleteQuietly(tar); + } + + ClusterIntegrationTestUtils.buildSegmentFromAvro(avroFiles.get(1), offlineTableConfig, schema, "_without_move", + _segmentDir, _tarDir); + jobSpec.setPushJobSpec(new PushJobSpec()); + + // Now test standalone tar push job runner + runner = new SegmentTarPushJobRunner(); + + Assert.assertEquals(dataDirSegments.listFiles().length, 1); + Assert.assertEquals(_tarDir.listFiles().length, 1); + + runner.init(jobSpec); + runner.run(); + + // should not see new segments in dataDir + //Assert.assertEquals(dataDirSegments.listFiles().length, 1); Review Comment: delete? ########## pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/BaseSegmentPushJobRunner.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.pinot.plugin.ingestion.batch.common; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.pinot.segment.local.utils.ConsistentDataPushUtils; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; +import org.apache.pinot.spi.ingestion.batch.runner.IngestionJobRunner; +import org.apache.pinot.spi.ingestion.batch.spec.PinotFSSpec; +import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; + +public abstract class BaseSegmentPushJobRunner implements IngestionJobRunner { + + protected SegmentGenerationJobSpec _spec; + protected String[] _files; + protected PinotFS _outputDirFS; + protected URI _outputDirURI; + protected List<String> _segmentsToPush = new ArrayList<>(); + protected Map<String, String> _segmentUriToTarPathMap; + protected boolean _consistentPushEnabled; + + /** + * Initialize BaseSegmentPushJobRunner with SegmentGenerationJobSpec + * Checks for required parameters in the spec and enablement of consistent data push. + */ + @Override + public void init(SegmentGenerationJobSpec spec) { + _spec = spec; + if (_spec.getPushJobSpec() == null) { + throw new RuntimeException("Missing PushJobSpec"); + } + + // Read Table spec + if (_spec.getTableSpec() == null) { + throw new RuntimeException("Missing tableSpec"); + } + + // Read Table config + if (_spec.getTableSpec().getTableConfigURI() == null) { + throw new RuntimeException("Missing property 'tableConfigURI' in 'tableSpec'"); + } + + _consistentPushEnabled = ConsistentDataPushUtils.consistentDataPushEnabled(_spec); + } + + /** + * Initialize filesystems and obtain the raw input files for upload. + */ + public void initFileSys() { + // init all file systems + List<PinotFSSpec> pinotFSSpecs = _spec.getPinotFSSpecs(); + for (PinotFSSpec pinotFSSpec : pinotFSSpecs) { + PinotFSFactory.register(pinotFSSpec.getScheme(), pinotFSSpec.getClassName(), new PinotConfiguration(pinotFSSpec)); + } + + // Get outputFS for writing output Pinot segments + try { + _outputDirURI = new URI(_spec.getOutputDirURI()); + if (_outputDirURI.getScheme() == null) { + _outputDirURI = new File(_spec.getOutputDirURI()).toURI(); + } + } catch (URISyntaxException e) { + throw new RuntimeException("outputDirURI is not valid - '" + _spec.getOutputDirURI() + "'"); + } + _outputDirFS = PinotFSFactory.create(_outputDirURI.getScheme()); + + // Get list of files to process + try { + _files = _outputDirFS.listFiles(_outputDirURI, true); + } catch (IOException e) { + throw new RuntimeException("Unable to list all files under outputDirURI - '" + _outputDirURI + "'"); + } + } + + /** + * Populates either _segmentsToPush or _segmentUriToTarPathMap based on push type. + */ + public abstract void getSegmentsToPush(); + + /** + * Returns segmentsTo based on segments obtained from getSegmentsToPush. + * The result will to be supplied to the segment replacement protocol when consistent data push is enabled. + */ + public abstract List<String> getSegmentsTo(); Review Comment: `getSegmentsToReplace()` ########## pinot-plugins/pinot-batch-ingestion/pinot-batch-ingestion-common/src/main/java/org/apache/pinot/plugin/ingestion/batch/common/BaseSegmentPushJobRunner.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.pinot.plugin.ingestion.batch.common; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.pinot.segment.local.utils.ConsistentDataPushUtils; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; +import org.apache.pinot.spi.ingestion.batch.runner.IngestionJobRunner; +import org.apache.pinot.spi.ingestion.batch.spec.PinotFSSpec; +import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; + +public abstract class BaseSegmentPushJobRunner implements IngestionJobRunner { + + protected SegmentGenerationJobSpec _spec; + protected String[] _files; + protected PinotFS _outputDirFS; + protected URI _outputDirURI; + protected List<String> _segmentsToPush = new ArrayList<>(); + protected Map<String, String> _segmentUriToTarPathMap; + protected boolean _consistentPushEnabled; + + /** + * Initialize BaseSegmentPushJobRunner with SegmentGenerationJobSpec + * Checks for required parameters in the spec and enablement of consistent data push. + */ + @Override + public void init(SegmentGenerationJobSpec spec) { + _spec = spec; + if (_spec.getPushJobSpec() == null) { + throw new RuntimeException("Missing PushJobSpec"); + } + + // Read Table spec + if (_spec.getTableSpec() == null) { + throw new RuntimeException("Missing tableSpec"); + } + + // Read Table config + if (_spec.getTableSpec().getTableConfigURI() == null) { + throw new RuntimeException("Missing property 'tableConfigURI' in 'tableSpec'"); + } + + _consistentPushEnabled = ConsistentDataPushUtils.consistentDataPushEnabled(_spec); + } + + /** + * Initialize filesystems and obtain the raw input files for upload. + */ + public void initFileSys() { + // init all file systems + List<PinotFSSpec> pinotFSSpecs = _spec.getPinotFSSpecs(); + for (PinotFSSpec pinotFSSpec : pinotFSSpecs) { + PinotFSFactory.register(pinotFSSpec.getScheme(), pinotFSSpec.getClassName(), new PinotConfiguration(pinotFSSpec)); + } + + // Get outputFS for writing output Pinot segments + try { + _outputDirURI = new URI(_spec.getOutputDirURI()); + if (_outputDirURI.getScheme() == null) { + _outputDirURI = new File(_spec.getOutputDirURI()).toURI(); + } + } catch (URISyntaxException e) { + throw new RuntimeException("outputDirURI is not valid - '" + _spec.getOutputDirURI() + "'"); + } + _outputDirFS = PinotFSFactory.create(_outputDirURI.getScheme()); + + // Get list of files to process + try { + _files = _outputDirFS.listFiles(_outputDirURI, true); + } catch (IOException e) { + throw new RuntimeException("Unable to list all files under outputDirURI - '" + _outputDirURI + "'"); + } + } + + /** + * Populates either _segmentsToPush or _segmentUriToTarPathMap based on push type. + */ + public abstract void getSegmentsToPush(); + + /** + * Returns segmentsTo based on segments obtained from getSegmentsToPush. + * The result will to be supplied to the segment replacement protocol when consistent data push is enabled. + */ + public abstract List<String> getSegmentsTo(); + + /** + * Upload segment obtained by getSegmentsToPush. + */ + public abstract void uploadSegments() + throws Exception; + + /** + * Runs the main logic of the segment push job runner. + * First initialize the filesystem, then upload the segments, while optionally configured to be wrapped around by + * the consistent data push protocol. + */ + @Override + public void run() { + initFileSys(); + Map<URI, String> uriToLineageEntryIdMap = null; + try { + getSegmentsToPush(); Review Comment: We can change this to ``` Map<String, String> segmentsUriToTarPathMap = SegmentPushUtils.getSegmentUriToTarPathMap(_outputDirURI, _pushJobSpec, _files) ... uploadSegments(segmentsUriToTarPathMap); ``` `Map<String, String> segmentsUriToTarPathMap` includes all required information. 1. tarPushJob needs the list of files directly return from `outputDirFS.listFiles()`. This is basically `segmentsUriToTarPathMap.values()` 2. UriPushJob needs the list of Uris and this is the same as `segmentsUriToTarPathMap.keys()` 3. MetadataPush needs the original map. With this, we don't need to enforce `getSegmentsToPush()` to be a void function with a bit weird requirement to set either `_segmentsToPush` or `_segmentUriToTarPathMap `. We can completely remove the `getSegmentsToPush()` abstract function. ########## pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/SegmentUploadIntegrationTest.java: ########## @@ -188,6 +202,133 @@ public void testUploadAndQuery() testCountStar(numDocs); } + /** + * Runs both SegmentMetadataPushJobRunner and SegmentTarPushJobRunner while enabling consistent data push. + * Checks that segments are properly loaded and segment lineage entry were also in expected states. + */ + @Test + public void testUploadAndQueryWithConsistentPush() + throws Exception { + // Create and upload the schema and table config + Schema schema = createSchema(); + addSchema(schema); + TableConfig offlineTableConfig = createOfflineTableConfigWithConsistentPush(); + addTableConfig(offlineTableConfig); + + List<File> avroFiles = getAllAvroFiles(); + + ClusterIntegrationTestUtils.buildSegmentFromAvro(avroFiles.get(0), offlineTableConfig, schema, "_with_move", + _segmentDir, _tarDir); + + // First test standalone metadata push job runner + BaseSegmentPushJobRunner runner = new SegmentMetadataPushJobRunner(); + SegmentGenerationJobSpec jobSpec = new SegmentGenerationJobSpec(); + PushJobSpec pushJobSpec = new PushJobSpec(); + pushJobSpec.setCopyToDeepStoreForMetadataPush(true); + jobSpec.setPushJobSpec(pushJobSpec); + PinotFSSpec fsSpec = new PinotFSSpec(); + fsSpec.setScheme("file"); + fsSpec.setClassName("org.apache.pinot.spi.filesystem.LocalPinotFS"); + jobSpec.setPinotFSSpecs(Lists.newArrayList(fsSpec)); + jobSpec.setOutputDirURI(_tarDir.getAbsolutePath()); + TableSpec tableSpec = new TableSpec(); + tableSpec.setTableName(DEFAULT_TABLE_NAME); + tableSpec.setTableConfigURI(_controllerRequestURLBuilder.forUpdateTableConfig(DEFAULT_TABLE_NAME)); + jobSpec.setTableSpec(tableSpec); + PinotClusterSpec clusterSpec = new PinotClusterSpec(); + clusterSpec.setControllerURI(_controllerBaseApiUrl); + jobSpec.setPinotClusterSpecs(new PinotClusterSpec[]{clusterSpec}); + + File dataDir = new File(_controllerConfig.getDataDir()); + File dataDirSegments = new File(dataDir, DEFAULT_TABLE_NAME); + + Assert.assertEquals(_tarDir.listFiles().length, 1); + + runner.init(jobSpec); + runner.run(); + + // Segment should be seen in dataDir + Assert.assertTrue(dataDirSegments.exists()); + Assert.assertEquals(dataDirSegments.listFiles().length, 1); + Assert.assertEquals(_tarDir.listFiles().length, 1); + + // test segment loaded + JsonNode segmentsList = getSegmentsList(); + Assert.assertEquals(segmentsList.size(), 1); + String segmentNameWithMove = segmentsList.get(0).asText(); Review Comment: What do you mean by `segmentNameWithMove` and `segmentNameWithoutMove`? It looks like you are appending the postfix to differentiate segment names. In this case, can we pass `System.currentTimeMillis()` as the postfix and we check whether the segment ends with the timestamp? ########## pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/ConsistentDataPushUtils.java: ########## @@ -0,0 +1,334 @@ +/** + * 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.segment.local.utils; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import javax.ws.rs.core.Response; +import org.apache.pinot.common.auth.AuthProviderUtils; +import org.apache.pinot.common.exception.HttpErrorStatusException; +import org.apache.pinot.common.restlet.resources.StartReplaceSegmentsRequest; +import org.apache.pinot.common.segment.generation.SegmentGenerationUtils; +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.spi.auth.AuthProvider; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.ingestion.batch.spec.Constants; +import org.apache.pinot.spi.ingestion.batch.spec.PinotClusterSpec; +import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; +import org.apache.pinot.spi.ingestion.batch.spec.SegmentNameGeneratorSpec; +import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.apache.pinot.spi.utils.JsonUtils; +import org.apache.pinot.spi.utils.retry.RetryPolicies; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ConsistentDataPushUtils { + private ConsistentDataPushUtils() { + } + + private static final Logger LOGGER = LoggerFactory.getLogger(SegmentPushUtils.class); + private static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + public static final String SEGMENT_NAME_POSTFIX = "segment.name.postfix"; + + /** + * Checks for enablement of consistent data push. If enabled, fetch the list of segments to be replaced, then + * invoke startReplaceSegments API and returns a map of controller URI to lineage entry IDs. + * If not, returns an empty hashmap. + */ + public static Map<URI, String> preUpload(SegmentGenerationJobSpec spec, List<String> segmentsTo) + throws Exception { + String rawTableName = spec.getTableSpec().getTableName(); + boolean consistentDataPushEnabled = consistentDataPushEnabled(spec); + LOGGER.info("Consistent data push is: {}", consistentDataPushEnabled ? "enabled" : "disabled"); + Map<URI, String> uriToLineageEntryIdMap = null; + if (consistentDataPushEnabled) { + LOGGER.info("Start consistent push for table: " + rawTableName); + Map<URI, List<String>> uriToExistingOfflineSegments = getSegmentsToReplace(spec, rawTableName); + LOGGER.info("Existing segments for table {}: " + uriToExistingOfflineSegments, rawTableName); + LOGGER.info("New segments for table: {}: " + segmentsTo, rawTableName); + uriToLineageEntryIdMap = startReplaceSegments(spec, uriToExistingOfflineSegments, segmentsTo); + } + return uriToLineageEntryIdMap; + } + + /** + * uriToLineageEntryIdMap is non-empty if and only if consistent data push is enabled. + * If uriToLineageEntryIdMap is non-empty, end the consistent data push protocol for each controller. + */ + public static void postUpload(SegmentGenerationJobSpec spec, Map<URI, String> uriToLineageEntryIdMap) { + String rawTableName = spec.getTableSpec().getTableName(); + if (uriToLineageEntryIdMap != null && !uriToLineageEntryIdMap.isEmpty()) { + LOGGER.info("End consistent push for table: " + rawTableName); + endReplaceSegments(spec, uriToLineageEntryIdMap); + } + } + + /** + * Builds a map of controller URI to startReplaceSegments URI for each Pinot cluster in the spec. + */ + public static Map<URI, URI> getStartReplaceSegmentUris(SegmentGenerationJobSpec spec, String rawTableName) { + Map<URI, URI> baseUriToStartReplaceSegmentUriMap = new HashMap<>(); + for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { + URI controllerURI; + try { + controllerURI = new URI(pinotClusterSpec.getControllerURI()); + baseUriToStartReplaceSegmentUriMap.put(controllerURI, + FileUploadDownloadClient.getStartReplaceSegmentsURI(controllerURI, rawTableName, + TableType.OFFLINE.toString(), true)); + } catch (URISyntaxException e) { + throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); + } + } + return baseUriToStartReplaceSegmentUriMap; + } + + /** + * Starts consistent data push protocol for each Pinot cluster in the spec. + * Returns a map of controller URI to segment lineage entry ID. + */ + public static Map<URI, String> startReplaceSegments(SegmentGenerationJobSpec spec, + Map<URI, List<String>> uriToSegmentsFrom, List<String> segmentsTo) + throws Exception { + Map<URI, String> uriToLineageEntryIdMap = new HashMap<>(); + String rawTableName = spec.getTableSpec().getTableName(); + Map<URI, URI> segmentsUris = getStartReplaceSegmentUris(spec, rawTableName); + AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); + LOGGER.info("Start replace segment URIs: " + segmentsUris); + + int attempts = 1; + long retryWaitMs = 1000L; + + for (Map.Entry<URI, URI> entry : segmentsUris.entrySet()) { + URI controllerUri = entry.getKey(); + URI startSegmentUri = entry.getValue(); + List<String> segmentsFrom = uriToSegmentsFrom.get(controllerUri); + + if (!Collections.disjoint(segmentsFrom, segmentsTo)) { + String errorMsg = + String.format("Found same segment names when attempting to enable consistent push for table: %s", + rawTableName); + LOGGER.error("SegmentsFrom: {}", segmentsFrom); + LOGGER.error("SegmentsTo: {}", segmentsTo); + LOGGER.error(errorMsg); + throw new RuntimeException(errorMsg); + } + + StartReplaceSegmentsRequest startReplaceSegmentsRequest = + new StartReplaceSegmentsRequest(segmentsFrom, segmentsTo); + RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { + try { + SimpleHttpResponse response = + FILE_UPLOAD_DOWNLOAD_CLIENT.startReplaceSegments(startSegmentUri, startReplaceSegmentsRequest, + authProvider); + + String responseString = response.getResponse(); + LOGGER.info( + "Got response {}: {} while sending start replace segment request for table: {}, uploadURI: {}, request:" + + " {}", response.getStatusCode(), responseString, rawTableName, startSegmentUri, + startReplaceSegmentsRequest); + String segmentLineageEntryId = + JsonUtils.stringToJsonNode(responseString).get("segmentLineageEntryId").asText(); + uriToLineageEntryIdMap.put(controllerUri, segmentLineageEntryId); + return true; + } catch (SocketTimeoutException se) { + // In case of the timeout, we should re-try. + return false; + } catch (HttpErrorStatusException e) { + if (e.getStatusCode() >= 500) { + return false; + } else { + if (e.getStatusCode() == Response.Status.NOT_FOUND.getStatusCode()) { + LOGGER.error("Table: {} not found when sending request: {}", rawTableName, startSegmentUri); + } + throw e; + } + } + }); + } + return uriToLineageEntryIdMap; + } + + /** + * Ends consistent data push protocol for each Pinot cluster in the spec. + */ + public static void endReplaceSegments(SegmentGenerationJobSpec spec, Map<URI, String> uriToLineageEntryIdMap) { + AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); + String rawTableName = spec.getTableSpec().getTableName(); + for (URI uri : uriToLineageEntryIdMap.keySet()) { + String segmentLineageEntryId = uriToLineageEntryIdMap.get(uri); + try { + FILE_UPLOAD_DOWNLOAD_CLIENT.endReplaceSegments( + FileUploadDownloadClient.getEndReplaceSegmentsURI(uri, rawTableName, TableType.OFFLINE.toString(), + segmentLineageEntryId), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS, authProvider); + } catch (URISyntaxException e) { + throw new RuntimeException("Got invalid controller uri - '" + uri + "'"); + } catch (HttpErrorStatusException | IOException e) { + e.printStackTrace(); + } + } + } + + /** + * Revert segment lineage entry when exception gets caught. This revert request is called at best effort. + * If the revert call fails at this point, the next startReplaceSegment call will do the cleanup + * by marking the previous entry to "REVERTED" and cleaning up the leftover segments. + */ + public static void handleUploadException(SegmentGenerationJobSpec spec, Map<URI, String> uriToLineageEntryIdMap, + Exception exception) { + if (uriToLineageEntryIdMap != null) { + LOGGER.error("Exception when pushing segments. Marking segment lineage entry to 'REVERTED'.", exception); + String rawTableName = spec.getTableSpec().getTableName(); + for (Map.Entry<URI, String> entry : uriToLineageEntryIdMap.entrySet()) { + String segmentLineageEntryId = entry.getValue(); + try { + URI uri = FileUploadDownloadClient.getRevertReplaceSegmentsURI(entry.getKey(), rawTableName, + TableType.OFFLINE.name(), segmentLineageEntryId, true); + SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT.revertReplaceSegments(uri); + LOGGER.info("Got response {}: {} while sending revert replace segment request for table: {}, uploadURI: {}", + response.getStatusCode(), response.getResponse(), rawTableName, entry.getKey()); + } catch (URISyntaxException | HttpErrorStatusException | IOException e) { + LOGGER.error("Exception when sending revert replace segment request to controller: {} for table: {}", + entry.getKey(), rawTableName, e); + } + } + } + } + + /** + * Ensures that all files in tarFilePaths have the expected tar file extension and obtain segment names given + * tarFilePaths. + */ + public static List<String> getTarSegmentsTo(List<String> tarFilePaths) { + List<String> segmentsTo = new ArrayList<>(); + for (String tarFilePath : tarFilePaths) { + File tarFile = new File(tarFilePath); + String fileName = tarFile.getName(); + Preconditions.checkArgument(fileName.endsWith(Constants.TAR_GZ_FILE_EXT)); + String segmentName = getSegmentNameFromFilePath(fileName); + segmentsTo.add(segmentName); + } + return segmentsTo; + } + + /** + * Ensures that all URIs in segmentUris have the expected tar file extension and obtain segment names given + * segmentUris. + */ + public static List<String> getUriSegmentsTo(List<String> segmentUris) { + List<String> segmentsTo = new ArrayList<>(); + for (String segmentUri : segmentUris) { + Preconditions.checkArgument(segmentUri.endsWith(Constants.TAR_GZ_FILE_EXT)); + String segmentName = getSegmentNameFromFilePath(segmentUri); + segmentsTo.add(segmentName); + } + return segmentsTo; + } + + /** + * Ensures that all tarPaths in segmentUriToTarPathMap have the expected tar file extension and obtain segment names + * given tarPaths. + */ + public static List<String> getMetadataSegmentsTo(Map<String, String> segmentUriToTarPathMap) { Review Comment: What does it mean by `Metadata` here? -- 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: commits-unsubscr...@pinot.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org For additional commands, e-mail: commits-h...@pinot.apache.org