From a0a425af23888c9ab9107a45f9b2a5fcc9070af8 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Thu, 17 Nov 2022 11:29:03 +0530 Subject: [PATCH 01/16] Add code to push metadata in minion tasks --- ...aseMultipleSegmentsConversionExecutor.java | 118 +++++- .../plugin/minion/tasks/MinionPushUtils.java | 374 ++++++++++++++++++ .../pinot/plugin/minion/tasks/TaskUtils.java | 73 ++++ 3 files changed, 562 insertions(+), 3 deletions(-) create mode 100644 pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java create mode 100644 pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 5b64c5842026..4b60e6ea5fa2 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -21,8 +21,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.io.File; +import java.net.URI; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,8 +48,17 @@ import org.apache.pinot.minion.event.MinionEventObserver; import org.apache.pinot.minion.event.MinionEventObservers; import org.apache.pinot.minion.exception.TaskCancelledException; +import org.apache.pinot.plugin.minion.tasks.segmentgenerationandpush.SegmentGenerationAndPushTaskUtils; import org.apache.pinot.spi.auth.AuthProvider; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; +import org.apache.pinot.spi.ingestion.batch.spec.PinotClusterSpec; +import org.apache.pinot.spi.ingestion.batch.spec.PushJobSpec; +import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; +import org.apache.pinot.spi.ingestion.batch.spec.TableSpec; import org.apache.pinot.spi.utils.builder.TableNameBuilder; +import org.apache.pinot.spi.utils.retry.AttemptsExceededException; +import org.apache.pinot.spi.utils.retry.RetriableOperationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +77,10 @@ public abstract class BaseMultipleSegmentsConversionExecutor extends BaseTaskExe private static final Logger LOGGER = LoggerFactory.getLogger(BaseMultipleSegmentsConversionExecutor.class); private static final String CUSTOM_SEGMENT_UPLOAD_CONTEXT_LINEAGE_ENTRY_ID = "lineageEntryId"; + private static final int DEFUALT_PUSH_ATTEMPTS = 5; + private static final int DEFAULT_PUSH_PARALLELISM = 1; + private static final long DEFAULT_PUSH_RETRY_INTERVAL_MILLIS = 1000L; + protected MinionConf _minionConf; // Tracking finer grained progress status. @@ -242,6 +257,11 @@ public List executeTask(PinotTaskConfig pinotTaskConfig new BasicHeader(FileUploadDownloadClient.CustomHeaders.SEGMENT_ZK_METADATA_CUSTOM_MAP_MODIFIER, segmentZKMetadataCustomMapModifier.toJsonString()); + URI outputSegmentTarURI = moveSegmentToOutputPinotFS(pinotTaskConfig.getConfigs(), convertedTarredSegmentFile); + LOGGER.info("Moved generated segment from [{}] to location: [{}]", convertedTarredSegmentFile, outputSegmentTarURI); + + + //TODO: Pass on these headers to push segment call List
httpHeaders = new ArrayList<>(); httpHeaders.add(segmentZKMetadataCustomMapModifierHeader); httpHeaders.addAll(AuthProviderUtils.toRequestHeaders(authProvider)); @@ -253,9 +273,10 @@ public List executeTask(PinotTaskConfig pinotTaskConfig TableNameBuilder.extractRawTableName(tableNameWithType)); List parameters = Arrays.asList(enableParallelPushProtectionParameter, tableNameParameter); - SegmentConversionUtils - .uploadSegment(configs, httpHeaders, parameters, tableNameWithType, resultSegmentName, uploadURL, - convertedTarredSegmentFile); + pushSegment(tableNameWithType, pinotTaskConfig.getConfigs(), outputSegmentTarURI, httpHeaders, parameters); +// SegmentConversionUtils +// .uploadSegment(configs, httpHeaders, parameters, tableNameWithType, resultSegmentName, uploadURL, +// convertedTarredSegmentFile); if (!FileUtils.deleteQuietly(convertedTarredSegmentFile)) { LOGGER.warn("Failed to delete tarred converted segment: {}", convertedTarredSegmentFile.getAbsolutePath()); } @@ -276,6 +297,97 @@ public List executeTask(PinotTaskConfig pinotTaskConfig } } + private void pushSegment(String tableName, Map taskConfigs, URI outputSegmentTarURI, + List
headers, List parameters) throws Exception { + String pushMode = taskConfigs.get(BatchConfigProperties.PUSH_MODE); + LOGGER.info("Trying to push Pinot segment with push mode {} from {}", pushMode, outputSegmentTarURI); + + PushJobSpec pushJobSpec = new PushJobSpec(); + pushJobSpec.setPushAttempts(DEFUALT_PUSH_ATTEMPTS); + pushJobSpec.setPushParallelism(DEFAULT_PUSH_PARALLELISM); + pushJobSpec.setPushRetryIntervalMillis(DEFAULT_PUSH_RETRY_INTERVAL_MILLIS); + pushJobSpec.setSegmentUriPrefix(taskConfigs.get(BatchConfigProperties.PUSH_SEGMENT_URI_PREFIX)); + pushJobSpec.setSegmentUriSuffix(taskConfigs.get(BatchConfigProperties.PUSH_SEGMENT_URI_SUFFIX)); + + SegmentGenerationJobSpec spec = generatePushJobSpec(tableName, taskConfigs, pushJobSpec); + + URI outputSegmentDirURI = null; + if (taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { + outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); + } + try (PinotFS outputFileFS = TaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { + switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { + case TAR: + try (PinotFS pinotFS = TaskUtils.getLocalPinotFs()) { + MinionPushUtils.pushSegments(spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString()), headers, parameters); + } catch (RetriableOperationException | AttemptsExceededException e) { + throw new RuntimeException(e); + } + break; + case URI: + try { + List segmentUris = new ArrayList<>(); + URI updatedURI = MinionPushUtils.generateSegmentTarURI(outputSegmentDirURI, outputSegmentTarURI, + pushJobSpec.getSegmentUriPrefix(), pushJobSpec.getSegmentUriSuffix()); + segmentUris.add(updatedURI.toString()); + MinionPushUtils.sendSegmentUris(spec, segmentUris, headers, parameters); + } catch (RetriableOperationException | AttemptsExceededException e) { + throw new RuntimeException(e); + } + break; + case METADATA: + try { + Map segmentUriToTarPathMap = + MinionPushUtils.getSegmentUriToTarPathMap(outputSegmentDirURI, pushJobSpec, + new String[]{outputSegmentTarURI.toString()}); + MinionPushUtils.sendSegmentUriAndMetadata(spec, outputFileFS, segmentUriToTarPathMap, headers, parameters); + } catch (RetriableOperationException | AttemptsExceededException e) { + throw new RuntimeException(e); + } + break; + default: + throw new UnsupportedOperationException("Unrecognized push mode - " + pushMode); + } + } + } + + private SegmentGenerationJobSpec generatePushJobSpec(String tableName, Map taskConfigs, + PushJobSpec pushJobSpec) { + + TableSpec tableSpec = new TableSpec(); + tableSpec.setTableName(tableName); + + PinotClusterSpec pinotClusterSpec = new PinotClusterSpec(); + pinotClusterSpec.setControllerURI(taskConfigs.get(BatchConfigProperties.PUSH_CONTROLLER_URI)); + PinotClusterSpec[] pinotClusterSpecs = new PinotClusterSpec[]{pinotClusterSpec}; + + SegmentGenerationJobSpec spec = new SegmentGenerationJobSpec(); + spec.setPushJobSpec(pushJobSpec); + spec.setTableSpec(tableSpec); + spec.setPinotClusterSpecs(pinotClusterSpecs); + spec.setAuthToken(taskConfigs.get(BatchConfigProperties.AUTH_TOKEN)); + + return spec; + } + + private URI moveSegmentToOutputPinotFS(Map taskConfigs, File localSegmentTarFile) + throws Exception { + if (!taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { + return localSegmentTarFile.toURI(); + } + URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); + try (PinotFS outputFileFS = TaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { + URI outputSegmentTarURI = URI.create(outputSegmentDirURI + localSegmentTarFile.getName()); + if (!Boolean.parseBoolean(taskConfigs.get(BatchConfigProperties.OVERWRITE_OUTPUT)) && outputFileFS.exists( + outputSegmentDirURI)) { + LOGGER.warn("Not overwrite existing output segment tar file: {}", outputFileFS.exists(outputSegmentDirURI)); + } else { + outputFileFS.copyFromLocalFile(localSegmentTarFile, outputSegmentTarURI); + } + return outputSegmentTarURI; + } + } + // SegmentUploadContext holds the info to conduct certain actions // before and after uploading multiple segments. protected static class SegmentUploadContext { diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java new file mode 100644 index 000000000000..53cd9c8048c2 --- /dev/null +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java @@ -0,0 +1,374 @@ +/** + * 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.minion.tasks; + +import com.google.common.base.Preconditions; +import java.io.File; +import java.io.InputStream; +import java.io.Serializable; +import java.net.URI; +import java.net.URISyntaxException; +import java.nio.file.FileSystems; +import java.nio.file.PathMatcher; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.http.Header; +import org.apache.http.NameValuePair; +import org.apache.http.message.BasicHeader; +import org.apache.pinot.common.auth.AuthProviderUtils; +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.TarGzCompressionUtils; +import org.apache.pinot.common.utils.http.HttpClient; +import org.apache.pinot.segment.spi.V1Constants; +import org.apache.pinot.segment.spi.creator.name.SegmentNameUtils; +import org.apache.pinot.spi.auth.AuthProvider; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; +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.PushJobSpec; +import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; +import org.apache.pinot.spi.utils.retry.AttemptsExceededException; +import org.apache.pinot.spi.utils.retry.RetriableOperationException; +import org.apache.pinot.spi.utils.retry.RetryPolicies; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class MinionPushUtils implements Serializable { + private MinionPushUtils() { + } + + private static final Logger LOGGER = LoggerFactory.getLogger(MinionPushUtils.class); + private static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); + + public static URI generateSegmentTarURI(URI dirURI, URI fileURI, String prefix, String suffix) { + if (StringUtils.isEmpty(prefix) && StringUtils.isEmpty(suffix)) { + // In case the FS doesn't provide scheme or host, will fill it up from dirURI. + String scheme = fileURI.getScheme(); + if (StringUtils.isEmpty(fileURI.getScheme())) { + scheme = dirURI.getScheme(); + } + String host = fileURI.getHost(); + if (StringUtils.isEmpty(fileURI.getHost())) { + host = dirURI.getHost(); + } + int port = fileURI.getPort(); + if (port < 0) { + port = dirURI.getPort(); + } + try { + return new URI(scheme, fileURI.getUserInfo(), host, port, fileURI.getPath(), fileURI.getQuery(), + fileURI.getFragment()); + } catch (URISyntaxException e) { + LOGGER.warn("Unable to generate push uri based from dir URI: {} and file URI: {}, directly return file URI.", + dirURI, fileURI); + return fileURI; + } + } + return URI.create((prefix != null ? prefix : "") + fileURI.getRawPath() + (suffix != null ? suffix : "")); + } + + public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSystem, List tarFilePaths, + List
headers, List parameters) + throws RetriableOperationException, AttemptsExceededException { + String tableName = spec.getTableSpec().getTableName(); + TableType tableType = tableName.endsWith("_" + TableType.REALTIME.name()) ? TableType.REALTIME : TableType.OFFLINE; + boolean cleanUpOutputDir = spec.isCleanUpOutputDir(); + LOGGER.info("Start pushing segments: {}... to locations: {} for table {}", + Arrays.toString(tarFilePaths.subList(0, Math.min(5, tarFilePaths.size())).toArray()), + Arrays.toString(spec.getPinotClusterSpecs()), tableName); + for (String tarFilePath : tarFilePaths) { + URI tarFileURI = URI.create(tarFilePath); + File tarFile = new File(tarFilePath); + String fileName = tarFile.getName(); + Preconditions.checkArgument(fileName.endsWith(Constants.TAR_GZ_FILE_EXT)); + String segmentName = fileName.substring(0, fileName.length() - Constants.TAR_GZ_FILE_EXT.length()); + for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { + URI controllerURI; + try { + controllerURI = new URI(pinotClusterSpec.getControllerURI()); + } catch (URISyntaxException e) { + throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); + } + LOGGER.info("Pushing segment: {} to location: {} for table {}", segmentName, controllerURI, tableName); + int attempts = 1; + if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushAttempts() > 0) { + attempts = spec.getPushJobSpec().getPushAttempts(); + } + long retryWaitMs = 1000L; + if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushRetryIntervalMillis() > 0) { + retryWaitMs = spec.getPushJobSpec().getPushRetryIntervalMillis(); + } + RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { + try (InputStream inputStream = fileSystem.open(tarFileURI)) { + SimpleHttpResponse response = + FILE_UPLOAD_DOWNLOAD_CLIENT.uploadSegment(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), + segmentName, inputStream, headers, + parameters, tableName, tableType); + LOGGER.info("Response for pushing table {} segment {} to location {} - {}: {}", tableName, segmentName, + controllerURI, response.getStatusCode(), response.getResponse()); + return true; + } catch (HttpErrorStatusException e) { + int statusCode = e.getStatusCode(); + if (statusCode >= 500) { + // Temporary exception + LOGGER.warn("Caught temporary exception while pushing table: {} segment: {} to {}, will retry", tableName, + segmentName, controllerURI, e); + return false; + } else { + // Permanent exception + LOGGER.error("Caught permanent exception while pushing table: {} segment: {} to {}, won't retry", + tableName, segmentName, controllerURI, e); + throw e; + } + } finally { + if (cleanUpOutputDir) { + fileSystem.delete(tarFileURI, true); + } + } + }); + } + } + } + + public static void sendSegmentUris(SegmentGenerationJobSpec spec, List segmentUris, + List
headers, List parameters) + throws RetriableOperationException, AttemptsExceededException { + String tableName = spec.getTableSpec().getTableName(); + LOGGER.info("Start sending table {} segment URIs: {} to locations: {}", tableName, + Arrays.toString(segmentUris.subList(0, Math.min(5, segmentUris.size())).toArray()), + Arrays.toString(spec.getPinotClusterSpecs())); + for (String segmentUri : segmentUris) { + URI segmentURI = URI.create(segmentUri); + PinotFS outputDirFS = PinotFSFactory.create(segmentURI.getScheme()); + for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { + URI controllerURI; + try { + controllerURI = new URI(pinotClusterSpec.getControllerURI()); + } catch (URISyntaxException e) { + throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); + } + LOGGER.info("Sending table {} segment URI: {} to location: {} for ", tableName, segmentUri, controllerURI); + int attempts = 1; + if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushAttempts() > 0) { + attempts = spec.getPushJobSpec().getPushAttempts(); + } + long retryWaitMs = 1000L; + if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushRetryIntervalMillis() > 0) { + retryWaitMs = spec.getPushJobSpec().getPushRetryIntervalMillis(); + } + RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { + try { + SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT + .sendSegmentUri(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentUri, + headers, parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + LOGGER.info("Response for pushing table {} segment uri {} to location {} - {}: {}", tableName, segmentUri, + controllerURI, response.getStatusCode(), response.getResponse()); + return true; + } catch (HttpErrorStatusException e) { + int statusCode = e.getStatusCode(); + if (statusCode >= 500) { + // Temporary exception + LOGGER.warn("Caught temporary exception while pushing table: {} segment uri: {} to {}, will retry", + tableName, segmentUri, controllerURI, e); + return false; + } else { + // Permanent exception + LOGGER.error("Caught permanent exception while pushing table: {} segment uri: {} to {}, won't retry", + tableName, segmentUri, controllerURI, e); + throw e; + } + } finally { + if (spec.isCleanUpOutputDir()) { + outputDirFS.delete(segmentURI, true); + } + } + }); + } + } + } + + /** + * This method takes a map of segment downloadURI to corresponding tar file path, and push those segments in + * metadata mode. + * The steps are: + * 1. Download segment from tar file path; + * 2. Untar segment metadata and creation meta files from the tar file to a segment metadata directory; + * 3. Tar this segment metadata directory into a tar file + * 4. Generate a POST request with segmentDownloadURI in header to push tar file to Pinot controller. + * + * @param spec is the segment generation job spec + * @param fileSystem is the PinotFs used to copy segment tar file + * @param segmentUriToTarPathMap contains the map of segment DownloadURI to segment tar file path + * @throws Exception + */ + public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, PinotFS fileSystem, + Map segmentUriToTarPathMap, List
headers, List parameters) + throws Exception { + String tableName = spec.getTableSpec().getTableName(); + LOGGER.info("Start pushing segment metadata: {} to locations: {} for table {}", segmentUriToTarPathMap, + Arrays.toString(spec.getPinotClusterSpecs()), tableName); + for (String segmentUriPath : segmentUriToTarPathMap.keySet()) { + String tarFilePath = segmentUriToTarPathMap.get(segmentUriPath); + String fileName = new File(tarFilePath).getName(); + // segments stored in Pinot deep store do not have .tar.gz extension + String segmentName = fileName.endsWith(Constants.TAR_GZ_FILE_EXT) + ? fileName.substring(0, fileName.length() - Constants.TAR_GZ_FILE_EXT.length()) : fileName; + SegmentNameUtils.validatePartialOrFullSegmentName(segmentName); + File segmentMetadataFile = generateSegmentMetadataFile(fileSystem, URI.create(tarFilePath)); + try { + for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { + URI controllerURI; + try { + controllerURI = new URI(pinotClusterSpec.getControllerURI()); + } catch (URISyntaxException e) { + throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); + } + LOGGER.info("Pushing segment: {} to location: {} for table {}", segmentName, controllerURI, tableName); + int attempts = 1; + if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushAttempts() > 0) { + attempts = spec.getPushJobSpec().getPushAttempts(); + } + long retryWaitMs = 1000L; + if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushRetryIntervalMillis() > 0) { + retryWaitMs = spec.getPushJobSpec().getPushRetryIntervalMillis(); + } + RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { + try { + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentUriPath)); + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, + FileUploadDownloadClient.FileUploadType.METADATA.toString())); + if (spec.getPushJobSpec() != null) { + headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, + String.valueOf(spec.getPushJobSpec().getCopyToDeepStoreForMetadataPush()))); + } + + SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT + .uploadSegmentMetadata(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentName, + segmentMetadataFile, headers, parameters, + HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + LOGGER.info("Response for pushing table {} segment {} to location {} - {}: {}", tableName, segmentName, + controllerURI, response.getStatusCode(), response.getResponse()); + return true; + } catch (HttpErrorStatusException e) { + int statusCode = e.getStatusCode(); + if (statusCode >= 500) { + // Temporary exception + LOGGER + .warn("Caught temporary exception while pushing table: {} segment: {} to {}, will retry", tableName, + segmentName, controllerURI, e); + return false; + } else { + // Permanent exception + LOGGER.error("Caught permanent exception while pushing table: {} segment: {} to {}, won't retry", + tableName, segmentName, controllerURI, e); + throw e; + } + } + }); + } + } finally { + FileUtils.deleteQuietly(segmentMetadataFile); + } + } + } + + public static Map getSegmentUriToTarPathMap(URI outputDirURI, PushJobSpec pushSpec, + String[] files) { + Map segmentUriToTarPathMap = new HashMap<>(); + PathMatcher pushFilePathMatcher = null; + if (pushSpec.getPushFileNamePattern() != null) { + pushFilePathMatcher = FileSystems.getDefault().getPathMatcher(pushSpec.getPushFileNamePattern()); + } + + for (String file : files) { + if (pushFilePathMatcher != null) { + if (!pushFilePathMatcher.matches(Paths.get(file))) { + continue; + } + } + + URI uri = URI.create(file); + if (uri.getPath().endsWith(Constants.TAR_GZ_FILE_EXT)) { + URI updatedURI = MinionPushUtils.generateSegmentTarURI(outputDirURI, uri, pushSpec.getSegmentUriPrefix(), + pushSpec.getSegmentUriSuffix()); + segmentUriToTarPathMap.put(updatedURI.toString(), file); + } + } + return segmentUriToTarPathMap; + } + + /** + * Generate a segment metadata only tar file, which contains only metadata.properties and creation.meta file. + * The purpose of this is to create a lean tar to push to Pinot controller for adding segments without downloading + * the complete segment and untar the segment tarball. + * + * 1. Download segment tar file to temp dir; + * 2. Extract only metadata.properties and creation.meta files from the segment tar file; + * 3. Tar both files into a segment metadata file. + * + */ + private static File generateSegmentMetadataFile(PinotFS fileSystem, URI tarFileURI) + throws Exception { + String uuid = UUID.randomUUID().toString(); + File tarFile = + new File(FileUtils.getTempDirectory(), "segmentTar-" + uuid + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION); + File segmentMetadataDir = new File(FileUtils.getTempDirectory(), "segmentMetadataDir-" + uuid); + try { + fileSystem.copyToLocalFile(tarFileURI, tarFile); + if (segmentMetadataDir.exists()) { + FileUtils.forceDelete(segmentMetadataDir); + } + FileUtils.forceMkdir(segmentMetadataDir); + + // Extract metadata.properties + LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", tarFile, segmentMetadataDir); + TarGzCompressionUtils.untarOneFile(tarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, + new File(segmentMetadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); + + // Extract creation.meta + LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", tarFile, segmentMetadataDir); + TarGzCompressionUtils.untarOneFile(tarFile, V1Constants.SEGMENT_CREATION_META, + new File(segmentMetadataDir, V1Constants.SEGMENT_CREATION_META)); + + File segmentMetadataTarFile = new File(FileUtils.getTempDirectory(), + "segmentMetadata-" + uuid + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION); + if (segmentMetadataTarFile.exists()) { + FileUtils.forceDelete(segmentMetadataTarFile); + } + LOGGER.info("Trying to tar segment metadata dir [{}] to [{}]", segmentMetadataDir, segmentMetadataTarFile); + TarGzCompressionUtils.createTarGzFile(segmentMetadataDir, segmentMetadataTarFile); + return segmentMetadataTarFile; + } finally { + FileUtils.deleteQuietly(tarFile); + FileUtils.deleteQuietly(segmentMetadataDir); + } + } +} diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java new file mode 100644 index 000000000000..538fdbb13b8f --- /dev/null +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java @@ -0,0 +1,73 @@ +/** + * 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.minion.tasks; + +import java.net.URI; +import java.util.Map; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.filesystem.LocalPinotFS; +import org.apache.pinot.spi.filesystem.PinotFS; +import org.apache.pinot.spi.filesystem.PinotFSFactory; +import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; +import org.apache.pinot.spi.plugin.PluginManager; +import org.apache.pinot.spi.utils.IngestionConfigUtils; + + +public class TaskUtils { + private TaskUtils() { + } + + static PinotFS getInputPinotFS(Map taskConfigs, URI fileURI) + throws Exception { + String fileURIScheme = fileURI.getScheme(); + if (fileURIScheme == null) { + return new LocalPinotFS(); + } + // Try to create PinotFS using given Input FileSystem config always + String fsClass = taskConfigs.get(BatchConfigProperties.INPUT_FS_CLASS); + if (fsClass != null) { + PinotFS pinotFS = PluginManager.get().createInstance(fsClass); + PinotConfiguration fsProps = IngestionConfigUtils.getInputFsProps(taskConfigs); + pinotFS.init(fsProps); + return pinotFS; + } + return PinotFSFactory.create(fileURIScheme); + } + + static PinotFS getOutputPinotFS(Map taskConfigs, URI fileURI) + throws Exception { + String fileURIScheme = (fileURI == null) ? null : fileURI.getScheme(); + if (fileURIScheme == null) { + return new LocalPinotFS(); + } + // Try to create PinotFS using given Input FileSystem config always + String fsClass = taskConfigs.get(BatchConfigProperties.OUTPUT_FS_CLASS); + if (fsClass != null) { + PinotFS pinotFS = PluginManager.get().createInstance(fsClass); + PinotConfiguration fsProps = IngestionConfigUtils.getOutputFsProps(taskConfigs); + pinotFS.init(fsProps); + return pinotFS; + } + return PinotFSFactory.create(fileURIScheme); + } + + static PinotFS getLocalPinotFs() { + return new LocalPinotFS(); + } +} From 8944ea4905c5d52117dc691c211d7a4f0a367cd2 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Thu, 17 Nov 2022 13:47:13 +0530 Subject: [PATCH 02/16] Fix metadata push --- ...aseMultipleSegmentsConversionExecutor.java | 2 - .../mergerollup/MergeRollupTaskGenerator.java | 41 +++++++++++++++++-- 2 files changed, 38 insertions(+), 5 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 4b60e6ea5fa2..ffe0fed4623e 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -260,8 +260,6 @@ public List executeTask(PinotTaskConfig pinotTaskConfig URI outputSegmentTarURI = moveSegmentToOutputPinotFS(pinotTaskConfig.getConfigs(), convertedTarredSegmentFile); LOGGER.info("Moved generated segment from [{}] to location: [{}]", convertedTarredSegmentFile, outputSegmentTarURI); - - //TODO: Pass on these headers to push segment call List
httpHeaders = new ArrayList<>(); httpHeaders.add(segmentZKMetadataCustomMapModifierHeader); httpHeaders.addAll(AuthProviderUtils.toRequestHeaders(authProvider)); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java index 0983e9f4cdb4..4c551f4f074f 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java @@ -18,6 +18,8 @@ */ package org.apache.pinot.plugin.minion.tasks.mergerollup; +import java.net.URI; +import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -38,6 +40,7 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.minion.MergeRollupTaskMetadata; +import org.apache.pinot.common.segment.generation.SegmentGenerationUtils; import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator; import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; @@ -51,6 +54,7 @@ import org.apache.pinot.spi.config.table.SegmentPartitionConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.utils.IngestionConfigUtils; import org.apache.pinot.spi.utils.TimeUtils; import org.apache.pinot.spi.utils.builder.TableNameBuilder; @@ -105,6 +109,8 @@ public class MergeRollupTaskGenerator extends BaseTaskGenerator { private static final int DEFAULT_MAX_NUM_RECORDS_PER_TASK = 50_000_000; private static final int DEFAULT_NUM_PARALLEL_BUCKETS = 1; private static final String REFRESH = "REFRESH"; + private static final BatchConfigProperties.SegmentPushType DEFAULT_SEGMENT_PUSH_TYPE = + BatchConfigProperties.SegmentPushType.TAR; // This is the metric that keeps track of the task delay in the number of time buckets. For example, if we see this // number to be 7 and merge task is configured with "bucketTimePeriod = 1d", this means that we have 7 days of @@ -540,12 +546,12 @@ private List createPinotTaskConfigs(List sel List pinotTaskConfigs = new ArrayList<>(); for (int i = 0; i < segmentNamesList.size(); i++) { - Map configs = new HashMap<>(); + String downloadURL = StringUtils.join(downloadURLsList.get(i), MinionConstants.URL_SEPARATOR); + Map configs = getPushTaskConfig(mergeConfigs, downloadURL); configs.put(MinionConstants.TABLE_NAME_KEY, offlineTableName); configs.put(MinionConstants.SEGMENT_NAME_KEY, StringUtils.join(segmentNamesList.get(i), MinionConstants.SEGMENT_NAME_SEPARATOR)); - configs.put(MinionConstants.DOWNLOAD_URL_KEY, - StringUtils.join(downloadURLsList.get(i), MinionConstants.URL_SEPARATOR)); + configs.put(MinionConstants.DOWNLOAD_URL_KEY, downloadURL); configs.put(MinionConstants.UPLOAD_URL_KEY, _clusterInfoAccessor.getVipUrl() + "/segments"); configs.put(MinionConstants.ENABLE_REPLACE_SEGMENTS_KEY, "true"); @@ -571,6 +577,35 @@ private List createPinotTaskConfigs(List sel return pinotTaskConfigs; } + private Map getPushTaskConfig(Map batchConfigMap, String downloadUrl) { + + try { + URI downloadURI = URI.create(downloadUrl); + URI outputDirURI = null; + if (!downloadURI.getScheme().contentEquals("http")) { + String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); + outputDirURI = URI.create(outputDir); + } + String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); + + Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); + if (outputDirURI != null) { + URI outputSegmentDirURI = + SegmentGenerationUtils.getRelativeOutputPath(outputDirURI, downloadURI, outputDirURI); + singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); + } + if ((outputDirURI == null) || (pushMode == null)) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); + } else { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + } + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl()); + return singleFileGenerationTaskConfig; + } catch (Exception e) { + return batchConfigMap; + } + } + private long getMergeRollupTaskDelayInNumTimeBuckets(long watermarkMs, long maxEndTimeMsOfCurrentLevel, long bufferTimeMs, long bucketTimeMs) { if (watermarkMs == -1 || maxEndTimeMsOfCurrentLevel == Long.MIN_VALUE) { From 7280b64879ed23c86d07d25adaccaeab61721edb Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Thu, 17 Nov 2022 19:29:19 +0530 Subject: [PATCH 03/16] Add config for overwrite dir --- .../tests/MergeRollupMinionClusterIntegrationTest.java | 4 ++++ .../minion/tasks/mergerollup/MergeRollupTaskGenerator.java | 1 + 2 files changed, 5 insertions(+) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java index 5846eba2d312..899a834250fb 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java @@ -50,6 +50,7 @@ import org.apache.pinot.spi.config.table.TableTaskConfig; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; @@ -160,6 +161,7 @@ private TableTaskConfig getSingleLevelConcatTaskConfig() { tableTaskConfigs.put("100days.maxNumRecordsPerTask", "15000"); tableTaskConfigs.put("ActualElapsedTime.aggregationType", "min"); tableTaskConfigs.put("WeatherDelay.aggregationType", "sum"); + tableTaskConfigs.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); return new TableTaskConfig(Collections.singletonMap(MinionConstants.MergeRollupTask.TASK_TYPE, tableTaskConfigs)); } @@ -169,6 +171,7 @@ private TableTaskConfig getSingleLevelRollupTaskConfig() { tableTaskConfigs.put("150days.bufferTimePeriod", "1d"); tableTaskConfigs.put("150days.bucketTimePeriod", "150d"); tableTaskConfigs.put("150days.roundBucketTimePeriod", "7d"); + tableTaskConfigs.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); return new TableTaskConfig(Collections.singletonMap(MinionConstants.MergeRollupTask.TASK_TYPE, tableTaskConfigs)); } @@ -185,6 +188,7 @@ private TableTaskConfig getMultiLevelConcatTaskConfig() { tableTaskConfigs.put("90days.bucketTimePeriod", "90d"); tableTaskConfigs.put("90days.maxNumRecordsPerSegment", "100000"); tableTaskConfigs.put("90days.maxNumRecordsPerTask", "100000"); + tableTaskConfigs.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); return new TableTaskConfig(Collections.singletonMap(MinionConstants.MergeRollupTask.TASK_TYPE, tableTaskConfigs)); } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java index 4c551f4f074f..8632366c80e3 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java @@ -561,6 +561,7 @@ private List createPinotTaskConfigs(List sel } } + configs.put(BatchConfigProperties.OVERWRITE_OUTPUT, taskConfigs.get(BatchConfigProperties.OVERWRITE_OUTPUT)); configs.put(MergeRollupTask.MERGE_TYPE_KEY, mergeConfigs.get(MergeTask.MERGE_TYPE_KEY)); configs.put(MergeRollupTask.MERGE_LEVEL_KEY, mergeLevel); configs.put(MergeTask.PARTITION_BUCKET_TIME_PERIOD_KEY, mergeConfigs.get(MergeTask.BUCKET_TIME_PERIOD_KEY)); From ce6c8feb34bdefe0c4a2217f0b1cf99fcbffe43b Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Thu, 17 Nov 2022 22:10:16 +0530 Subject: [PATCH 04/16] Fix download url to generate outputDir --- ...rgeRollupMinionClusterIntegrationTest.java | 141 +++++++++++++++++- .../mergerollup/MergeRollupTaskGenerator.java | 48 +++--- 2 files changed, 165 insertions(+), 24 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java index 899a834250fb..d733b7529a44 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java @@ -71,6 +71,7 @@ public class MergeRollupMinionClusterIntegrationTest extends BaseClusterIntegrat private static final String SINGLE_LEVEL_CONCAT_TEST_TABLE = "myTable1"; private static final String SINGLE_LEVEL_ROLLUP_TEST_TABLE = "myTable2"; private static final String MULTI_LEVEL_CONCAT_TEST_TABLE = "myTable3"; + private static final String SINGLE_LEVEL_CONCAT_METADATA_TEST_TABLE = "myTable4"; private static final long TIMEOUT_IN_MS = 10_000L; protected PinotHelixTaskResourceManager _helixTaskResourceManager; @@ -80,15 +81,17 @@ public class MergeRollupMinionClusterIntegrationTest extends BaseClusterIntegrat protected final File _segmentDir1 = new File(_tempDir, "segmentDir1"); protected final File _segmentDir2 = new File(_tempDir, "segmentDir2"); protected final File _segmentDir3 = new File(_tempDir, "segmentDir3"); + protected final File _segmentDir4 = new File(_tempDir, "segmentDir4"); protected final File _tarDir1 = new File(_tempDir, "tarDir1"); protected final File _tarDir2 = new File(_tempDir, "tarDir2"); protected final File _tarDir3 = new File(_tempDir, "tarDir3"); + protected final File _tarDir4 = new File(_tempDir, "tarDir4"); @BeforeClass public void setUp() throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir1, _segmentDir2, _segmentDir3, _tarDir1, _tarDir2, - _tarDir3); + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir1, _segmentDir2, _segmentDir3, _segmentDir4, _tarDir1, _tarDir2, + _tarDir3, _tarDir4); // Start the Pinot cluster startZk(); @@ -106,9 +109,12 @@ public void setUp() getMultiColumnsSegmentPartitionConfig()); TableConfig multiLevelConcatTableConfig = createOfflineTableConfig(MULTI_LEVEL_CONCAT_TEST_TABLE, getMultiLevelConcatTaskConfig()); + TableConfig singleLevelConcatMetadataTableConfig = + createOfflineTableConfig(SINGLE_LEVEL_CONCAT_METADATA_TEST_TABLE, getSingleLevelConcatMetadataTaskConfig()); addTableConfig(singleLevelConcatTableConfig); addTableConfig(singleLevelRollupTableConfig); addTableConfig(multiLevelConcatTableConfig); + addTableConfig(singleLevelConcatMetadataTableConfig); // Unpack the Avro files List avroFiles = unpackAvroData(_tempDir); @@ -120,9 +126,13 @@ public void setUp() buildSegmentsFromAvroWithPostfix(avroFiles, singleLevelRollupTableConfig, schema, 0, _segmentDir2, _tarDir2, "2"); ClusterIntegrationTestUtils .buildSegmentsFromAvro(avroFiles, multiLevelConcatTableConfig, schema, 0, _segmentDir3, _tarDir3); + ClusterIntegrationTestUtils + .buildSegmentsFromAvro(avroFiles, singleLevelConcatMetadataTableConfig, schema, 0, _segmentDir4, _tarDir4); uploadSegments(SINGLE_LEVEL_CONCAT_TEST_TABLE, _tarDir1); uploadSegments(SINGLE_LEVEL_ROLLUP_TEST_TABLE, _tarDir2); uploadSegments(MULTI_LEVEL_CONCAT_TEST_TABLE, _tarDir3); + uploadSegments(SINGLE_LEVEL_CONCAT_METADATA_TEST_TABLE, _tarDir4); + // Set up the H2 connection setUpH2Connection(avroFiles); @@ -165,6 +175,20 @@ private TableTaskConfig getSingleLevelConcatTaskConfig() { return new TableTaskConfig(Collections.singletonMap(MinionConstants.MergeRollupTask.TASK_TYPE, tableTaskConfigs)); } + private TableTaskConfig getSingleLevelConcatMetadataTaskConfig() { + Map tableTaskConfigs = new HashMap<>(); + tableTaskConfigs.put("100days.mergeType", "concat"); + tableTaskConfigs.put("100days.bufferTimePeriod", "1d"); + tableTaskConfigs.put("100days.bucketTimePeriod", "100d"); + tableTaskConfigs.put("100days.maxNumRecordsPerSegment", "15000"); + tableTaskConfigs.put("100days.maxNumRecordsPerTask", "15000"); + tableTaskConfigs.put("ActualElapsedTime.aggregationType", "min"); + tableTaskConfigs.put("WeatherDelay.aggregationType", "sum"); + tableTaskConfigs.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); + tableTaskConfigs.put(BatchConfigProperties.PUSH_MODE, BatchConfigProperties.SegmentPushType.METADATA.toString()); + return new TableTaskConfig(Collections.singletonMap(MinionConstants.MergeRollupTask.TASK_TYPE, tableTaskConfigs)); + } + private TableTaskConfig getSingleLevelRollupTaskConfig() { Map tableTaskConfigs = new HashMap<>(); tableTaskConfigs.put("150days.mergeType", "rollup"); @@ -349,6 +373,119 @@ public void testSingleLevelConcat() verifyTableDelete(offlineTableName); } + /** + * Test single level concat task with maxNumRecordPerTask, maxNumRecordPerSegment constraints + * Push type is set to Metadata + */ + @Test + public void testSingleLevelConcatWithMetadataPush() + throws Exception { + // The original segments are time partitioned by month: + // segmentName (totalDocs) + // myTable1_16071_16101_3 (9746) + // myTable1_16102_16129_4 (8690) + // myTable1_16130_16159_5 (9621) + // myTable1_16160_16189_6 (9454) + // myTable1_16190_16220_7 (10329) + // myTable1_16221_16250_8 (10468) + // myTable1_16251_16281_9 (10499) + // myTable1_16282_16312_10 (10196) + // myTable1_16313_16342_11 (9136) + // myTable1_16343_16373_0 (9292) + // myTable1_16374_16404_1 (8736) + // myTable1_16405_16435_2 (9378) + + // Expected merge tasks and result segments: + // 1. + // {myTable1_16071_16101_3} + // -> {merged_100days_T1_0_myTable1_16071_16099_0, merged_100days_T1_0_myTable1_16100_16101_1} + // 2. + // {merged_100days_T1_0_myTable1_16100_16101_1, myTable1_16102_16129_4, myTable1_16130_16159_5} + // -> {merged_100days_T2_0_myTable1_16100_???_0(15000), merged_100days_T2_0_myTable1_???_16159_1} + // {myTable1_16160_16189_6, myTable1_16190_16220_7} + // -> {merged_100days_T2_1_myTable1_16160_16199_0, merged_100days_T2_1_myTable1_16200_16220_1} + // 3. + // {merged_100days_T2_1_myTable1_16200_16220_1, myTable1_16221_16250_8} + // -> {merged_100days_T3_0_myTable1_16200_???_0(15000), merged_100days_T3_0_myTable1_???_16250_1} + // {myTable1_16251_16281_9, myTable1_16282_16312_10} + // -> {merged_100days_T3_1_myTable1_16251_???_0(15000), merged_100days_T3_1_myTable1_???_16299_1, + // merged_100days_T3_1_myTable1_16300_16312_2} + // 4. + // {merged_100days_T3_1_myTable1_16300_16312_2, myTable1_16313_16342_11, myTable1_16343_16373_0} + // -> {merged_100days_T4_0_myTable1_16300_???_0(15000), merged_100days_T4_0_myTable1_???_16373_1} + // {myTable1_16374_16404_1} + // -> {merged_100days_T4_1_16374_16399_0, merged_100days_T4_1_16400_16404_1} + // 5. + // {merged_100days_T4_1_16400_16404_1, myTable1_16405_16435_2} + // -> {merged_100days_T5_0_myTable1_16400_16435_0} + + String sqlQuery = "SELECT count(*) FROM myTable4"; // 115545 rows for the test table + JsonNode expectedJson = postQuery(sqlQuery, _brokerBaseApiUrl); + int[] expectedNumSubTasks = {1, 2, 2, 2, 1}; + int[] expectedNumSegmentsQueried = {13, 12, 13, 13, 12}; + long expectedWatermark = 16000 * 86_400_000L; + String offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(SINGLE_LEVEL_CONCAT_METADATA_TEST_TABLE); + int numTasks = 0; + for (String tasks = _taskManager.scheduleTasks(offlineTableName).get(MinionConstants.MergeRollupTask.TASK_TYPE); + tasks != null; tasks = + _taskManager.scheduleTasks(offlineTableName).get(MinionConstants.MergeRollupTask.TASK_TYPE), numTasks++) { + assertEquals(_helixTaskResourceManager.getSubtaskConfigs(tasks).size(), expectedNumSubTasks[numTasks]); + assertTrue(_helixTaskResourceManager.getTaskQueues() + .contains(PinotHelixTaskResourceManager.getHelixJobQueueName(MinionConstants.MergeRollupTask.TASK_TYPE))); + // Will not schedule task if there's incomplete task + assertNull( + _taskManager.scheduleTasks(offlineTableName).get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); + waitForTaskToComplete(); + + // Check watermark + MergeRollupTaskMetadata minionTaskMetadata = MergeRollupTaskMetadata.fromZNRecord( + _taskManager.getClusterInfoAccessor() + .getMinionTaskMetadataZNRecord(MinionConstants.MergeRollupTask.TASK_TYPE, offlineTableName)); + assertNotNull(minionTaskMetadata); + assertEquals((long) minionTaskMetadata.getWatermarkMap().get("100days"), expectedWatermark); + expectedWatermark += 100 * 86_400_000L; + + // Check metadata of merged segments + for (SegmentZKMetadata metadata : _pinotHelixResourceManager.getSegmentsZKMetadata(offlineTableName)) { + if (metadata.getSegmentName().startsWith("merged")) { + // Check merged segment zk metadata + assertNotNull(metadata.getCustomMap()); + assertEquals("100days", + metadata.getCustomMap().get(MinionConstants.MergeRollupTask.SEGMENT_ZK_METADATA_MERGE_LEVEL_KEY)); + // Check merged segments are time partitioned + assertEquals(metadata.getEndTimeMs() / (86_400_000L * 100), metadata.getStartTimeMs() / (86_400_000L * 100)); + } + } + + final int finalNumTasks = numTasks; + TestUtils.waitForCondition(aVoid -> { + try { + // Check num total doc of merged segments are the same as the original segments + JsonNode actualJson = postQuery(sqlQuery, _brokerBaseApiUrl); + if (!SqlResultComparator.areEqual(actualJson, expectedJson, sqlQuery)) { + return false; + } + // Check query routing + int numSegmentsQueried = actualJson.get("numSegmentsQueried").asInt(); + return numSegmentsQueried == expectedNumSegmentsQueried[finalNumTasks]; + } catch (Exception e) { + throw new RuntimeException(e); + } + }, TIMEOUT_IN_MS, "Timeout while validating segments"); + } + // Check total tasks + assertEquals(numTasks, 5); + + assertTrue(_controllerStarter.getControllerMetrics() + .containsGauge("mergeRollupTaskDelayInNumBuckets.myTable4_OFFLINE.100days")); + + // Drop the table + dropOfflineTable(SINGLE_LEVEL_CONCAT_METADATA_TEST_TABLE); + + // Check if the task metadata is cleaned up on table deletion + verifyTableDelete(offlineTableName); + } + /** * Test single level rollup task with duplicate data (original segments * 2) */ diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java index 8632366c80e3..15ed1ba06e14 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java @@ -547,7 +547,7 @@ private List createPinotTaskConfigs(List sel List pinotTaskConfigs = new ArrayList<>(); for (int i = 0; i < segmentNamesList.size(); i++) { String downloadURL = StringUtils.join(downloadURLsList.get(i), MinionConstants.URL_SEPARATOR); - Map configs = getPushTaskConfig(mergeConfigs, downloadURL); + Map configs = getPushTaskConfig(taskConfigs, downloadURL); configs.put(MinionConstants.TABLE_NAME_KEY, offlineTableName); configs.put(MinionConstants.SEGMENT_NAME_KEY, StringUtils.join(segmentNamesList.get(i), MinionConstants.SEGMENT_NAME_SEPARATOR)); @@ -578,33 +578,37 @@ private List createPinotTaskConfigs(List sel return pinotTaskConfigs; } - private Map getPushTaskConfig(Map batchConfigMap, String downloadUrl) { - + private Map getPushTaskConfig(Map batchConfigMap, String downloadUrls) { try { - URI downloadURI = URI.create(downloadUrl); - URI outputDirURI = null; - if (!downloadURI.getScheme().contentEquals("http")) { - String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); - outputDirURI = URI.create(outputDir); - } - String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); + String[] downloadURLList = downloadUrls.split(MinionConstants.SEGMENT_NAME_SEPARATOR); + if (downloadURLList.length > 0) { + String downloadUrl = downloadURLList[0]; + URI downloadURI = URI.create(downloadUrl); + URI outputDirURI = null; + if (!downloadURI.getScheme().contentEquals("http")) { + String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); + outputDirURI = URI.create(outputDir); + } + String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); - Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); - if (outputDirURI != null) { - URI outputSegmentDirURI = - SegmentGenerationUtils.getRelativeOutputPath(outputDirURI, downloadURI, outputDirURI); - singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); - } - if ((outputDirURI == null) || (pushMode == null)) { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); - } else { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); + if (outputDirURI != null) { + URI outputSegmentDirURI = SegmentGenerationUtils.getRelativeOutputPath(outputDirURI, downloadURI, outputDirURI); + singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); + } + if ((outputDirURI == null) || (pushMode == null)) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); + } else { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + } + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl()); + return singleFileGenerationTaskConfig; } - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl()); - return singleFileGenerationTaskConfig; + return batchConfigMap; } catch (Exception e) { return batchConfigMap; } + } private long getMergeRollupTaskDelayInNumTimeBuckets(long watermarkMs, long maxEndTimeMsOfCurrentLevel, From 0cd020b3486cfd1568fdfe78095c1c836b9d296c Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Thu, 17 Nov 2022 22:16:37 +0530 Subject: [PATCH 05/16] Fix linting --- .../tests/MergeRollupMinionClusterIntegrationTest.java | 5 +++-- .../tasks/BaseMultipleSegmentsConversionExecutor.java | 8 ++++---- .../apache/pinot/plugin/minion/tasks/MinionPushUtils.java | 3 --- .../tasks/mergerollup/MergeRollupTaskGenerator.java | 8 ++++---- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java index d733b7529a44..3c27188b7fa6 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java @@ -90,8 +90,9 @@ public class MergeRollupMinionClusterIntegrationTest extends BaseClusterIntegrat @BeforeClass public void setUp() throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir1, _segmentDir2, _segmentDir3, _segmentDir4, _tarDir1, _tarDir2, - _tarDir3, _tarDir4); + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, + _segmentDir1, _segmentDir2, _segmentDir3, _segmentDir4, + _tarDir1, _tarDir2, _tarDir3, _tarDir4); // Start the Pinot cluster startZk(); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index ffe0fed4623e..d108702c6c13 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -24,7 +24,6 @@ import java.net.URI; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -48,7 +47,6 @@ import org.apache.pinot.minion.event.MinionEventObserver; import org.apache.pinot.minion.event.MinionEventObservers; import org.apache.pinot.minion.exception.TaskCancelledException; -import org.apache.pinot.plugin.minion.tasks.segmentgenerationandpush.SegmentGenerationAndPushTaskUtils; import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.filesystem.PinotFS; import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; @@ -258,7 +256,8 @@ public List executeTask(PinotTaskConfig pinotTaskConfig segmentZKMetadataCustomMapModifier.toJsonString()); URI outputSegmentTarURI = moveSegmentToOutputPinotFS(pinotTaskConfig.getConfigs(), convertedTarredSegmentFile); - LOGGER.info("Moved generated segment from [{}] to location: [{}]", convertedTarredSegmentFile, outputSegmentTarURI); + LOGGER.info("Moved generated segment from [{}] to location: [{}]", + convertedTarredSegmentFile, outputSegmentTarURI); List
httpHeaders = new ArrayList<>(); httpHeaders.add(segmentZKMetadataCustomMapModifierHeader); @@ -317,7 +316,8 @@ private void pushSegment(String tableName, Map taskConfigs, URI switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { case TAR: try (PinotFS pinotFS = TaskUtils.getLocalPinotFs()) { - MinionPushUtils.pushSegments(spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString()), headers, parameters); + MinionPushUtils.pushSegments( + spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString()), headers, parameters); } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java index 53cd9c8048c2..1a8343a360d7 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java @@ -27,7 +27,6 @@ import java.nio.file.FileSystems; import java.nio.file.PathMatcher; import java.nio.file.Paths; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -38,7 +37,6 @@ import org.apache.http.Header; import org.apache.http.NameValuePair; import org.apache.http.message.BasicHeader; -import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; import org.apache.pinot.common.utils.FileUploadDownloadClient; import org.apache.pinot.common.utils.SimpleHttpResponse; @@ -46,7 +44,6 @@ import org.apache.pinot.common.utils.http.HttpClient; import org.apache.pinot.segment.spi.V1Constants; import org.apache.pinot.segment.spi.creator.name.SegmentNameUtils; -import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.filesystem.PinotFS; import org.apache.pinot.spi.filesystem.PinotFSFactory; diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java index 15ed1ba06e14..8cdade2de8d7 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java @@ -19,7 +19,6 @@ package org.apache.pinot.plugin.minion.tasks.mergerollup; import java.net.URI; -import java.net.URISyntaxException; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -593,8 +592,10 @@ private Map getPushTaskConfig(Map batchConfigMap Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); if (outputDirURI != null) { - URI outputSegmentDirURI = SegmentGenerationUtils.getRelativeOutputPath(outputDirURI, downloadURI, outputDirURI); - singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); + URI outputSegmentDirURI = SegmentGenerationUtils.getRelativeOutputPath( + outputDirURI, downloadURI, outputDirURI); + singleFileGenerationTaskConfig.put( + BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); } if ((outputDirURI == null) || (pushMode == null)) { singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); @@ -608,7 +609,6 @@ private Map getPushTaskConfig(Map batchConfigMap } catch (Exception e) { return batchConfigMap; } - } private long getMergeRollupTaskDelayInNumTimeBuckets(long watermarkMs, long maxEndTimeMsOfCurrentLevel, From 2bed0e64ae8dd377a24e457bf5f2ee288c3eda25 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 18 Nov 2022 10:46:00 +0530 Subject: [PATCH 06/16] Remove MinionPushUtils class --- ...aseMultipleSegmentsConversionExecutor.java | 11 +- .../plugin/minion/tasks/MinionPushUtils.java | 371 ------------------ .../segment/local/utils/SegmentPushUtils.java | 74 +++- 3 files changed, 61 insertions(+), 395 deletions(-) delete mode 100644 pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index d108702c6c13..507bf6f67cff 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -47,6 +47,7 @@ import org.apache.pinot.minion.event.MinionEventObserver; import org.apache.pinot.minion.event.MinionEventObservers; import org.apache.pinot.minion.exception.TaskCancelledException; +import org.apache.pinot.segment.local.utils.SegmentPushUtils; import org.apache.pinot.spi.auth.AuthProvider; import org.apache.pinot.spi.filesystem.PinotFS; import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; @@ -316,7 +317,7 @@ private void pushSegment(String tableName, Map taskConfigs, URI switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { case TAR: try (PinotFS pinotFS = TaskUtils.getLocalPinotFs()) { - MinionPushUtils.pushSegments( + SegmentPushUtils.pushSegments( spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString()), headers, parameters); } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); @@ -325,10 +326,10 @@ private void pushSegment(String tableName, Map taskConfigs, URI case URI: try { List segmentUris = new ArrayList<>(); - URI updatedURI = MinionPushUtils.generateSegmentTarURI(outputSegmentDirURI, outputSegmentTarURI, + URI updatedURI = SegmentPushUtils.generateSegmentTarURI(outputSegmentDirURI, outputSegmentTarURI, pushJobSpec.getSegmentUriPrefix(), pushJobSpec.getSegmentUriSuffix()); segmentUris.add(updatedURI.toString()); - MinionPushUtils.sendSegmentUris(spec, segmentUris, headers, parameters); + SegmentPushUtils.sendSegmentUris(spec, segmentUris, headers, parameters); } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); } @@ -336,9 +337,9 @@ private void pushSegment(String tableName, Map taskConfigs, URI case METADATA: try { Map segmentUriToTarPathMap = - MinionPushUtils.getSegmentUriToTarPathMap(outputSegmentDirURI, pushJobSpec, + SegmentPushUtils.getSegmentUriToTarPathMap(outputSegmentDirURI, pushJobSpec, new String[]{outputSegmentTarURI.toString()}); - MinionPushUtils.sendSegmentUriAndMetadata(spec, outputFileFS, segmentUriToTarPathMap, headers, parameters); + SegmentPushUtils.sendSegmentUriAndMetadata(spec, outputFileFS, segmentUriToTarPathMap, headers, parameters); } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java deleted file mode 100644 index 1a8343a360d7..000000000000 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionPushUtils.java +++ /dev/null @@ -1,371 +0,0 @@ -/** - * 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.minion.tasks; - -import com.google.common.base.Preconditions; -import java.io.File; -import java.io.InputStream; -import java.io.Serializable; -import java.net.URI; -import java.net.URISyntaxException; -import java.nio.file.FileSystems; -import java.nio.file.PathMatcher; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import org.apache.commons.io.FileUtils; -import org.apache.commons.lang3.StringUtils; -import org.apache.http.Header; -import org.apache.http.NameValuePair; -import org.apache.http.message.BasicHeader; -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.TarGzCompressionUtils; -import org.apache.pinot.common.utils.http.HttpClient; -import org.apache.pinot.segment.spi.V1Constants; -import org.apache.pinot.segment.spi.creator.name.SegmentNameUtils; -import org.apache.pinot.spi.config.table.TableType; -import org.apache.pinot.spi.filesystem.PinotFS; -import org.apache.pinot.spi.filesystem.PinotFSFactory; -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.PushJobSpec; -import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; -import org.apache.pinot.spi.utils.retry.AttemptsExceededException; -import org.apache.pinot.spi.utils.retry.RetriableOperationException; -import org.apache.pinot.spi.utils.retry.RetryPolicies; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class MinionPushUtils implements Serializable { - private MinionPushUtils() { - } - - private static final Logger LOGGER = LoggerFactory.getLogger(MinionPushUtils.class); - private static final FileUploadDownloadClient FILE_UPLOAD_DOWNLOAD_CLIENT = new FileUploadDownloadClient(); - - public static URI generateSegmentTarURI(URI dirURI, URI fileURI, String prefix, String suffix) { - if (StringUtils.isEmpty(prefix) && StringUtils.isEmpty(suffix)) { - // In case the FS doesn't provide scheme or host, will fill it up from dirURI. - String scheme = fileURI.getScheme(); - if (StringUtils.isEmpty(fileURI.getScheme())) { - scheme = dirURI.getScheme(); - } - String host = fileURI.getHost(); - if (StringUtils.isEmpty(fileURI.getHost())) { - host = dirURI.getHost(); - } - int port = fileURI.getPort(); - if (port < 0) { - port = dirURI.getPort(); - } - try { - return new URI(scheme, fileURI.getUserInfo(), host, port, fileURI.getPath(), fileURI.getQuery(), - fileURI.getFragment()); - } catch (URISyntaxException e) { - LOGGER.warn("Unable to generate push uri based from dir URI: {} and file URI: {}, directly return file URI.", - dirURI, fileURI); - return fileURI; - } - } - return URI.create((prefix != null ? prefix : "") + fileURI.getRawPath() + (suffix != null ? suffix : "")); - } - - public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSystem, List tarFilePaths, - List
headers, List parameters) - throws RetriableOperationException, AttemptsExceededException { - String tableName = spec.getTableSpec().getTableName(); - TableType tableType = tableName.endsWith("_" + TableType.REALTIME.name()) ? TableType.REALTIME : TableType.OFFLINE; - boolean cleanUpOutputDir = spec.isCleanUpOutputDir(); - LOGGER.info("Start pushing segments: {}... to locations: {} for table {}", - Arrays.toString(tarFilePaths.subList(0, Math.min(5, tarFilePaths.size())).toArray()), - Arrays.toString(spec.getPinotClusterSpecs()), tableName); - for (String tarFilePath : tarFilePaths) { - URI tarFileURI = URI.create(tarFilePath); - File tarFile = new File(tarFilePath); - String fileName = tarFile.getName(); - Preconditions.checkArgument(fileName.endsWith(Constants.TAR_GZ_FILE_EXT)); - String segmentName = fileName.substring(0, fileName.length() - Constants.TAR_GZ_FILE_EXT.length()); - for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { - URI controllerURI; - try { - controllerURI = new URI(pinotClusterSpec.getControllerURI()); - } catch (URISyntaxException e) { - throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); - } - LOGGER.info("Pushing segment: {} to location: {} for table {}", segmentName, controllerURI, tableName); - int attempts = 1; - if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushAttempts() > 0) { - attempts = spec.getPushJobSpec().getPushAttempts(); - } - long retryWaitMs = 1000L; - if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushRetryIntervalMillis() > 0) { - retryWaitMs = spec.getPushJobSpec().getPushRetryIntervalMillis(); - } - RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { - try (InputStream inputStream = fileSystem.open(tarFileURI)) { - SimpleHttpResponse response = - FILE_UPLOAD_DOWNLOAD_CLIENT.uploadSegment(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), - segmentName, inputStream, headers, - parameters, tableName, tableType); - LOGGER.info("Response for pushing table {} segment {} to location {} - {}: {}", tableName, segmentName, - controllerURI, response.getStatusCode(), response.getResponse()); - return true; - } catch (HttpErrorStatusException e) { - int statusCode = e.getStatusCode(); - if (statusCode >= 500) { - // Temporary exception - LOGGER.warn("Caught temporary exception while pushing table: {} segment: {} to {}, will retry", tableName, - segmentName, controllerURI, e); - return false; - } else { - // Permanent exception - LOGGER.error("Caught permanent exception while pushing table: {} segment: {} to {}, won't retry", - tableName, segmentName, controllerURI, e); - throw e; - } - } finally { - if (cleanUpOutputDir) { - fileSystem.delete(tarFileURI, true); - } - } - }); - } - } - } - - public static void sendSegmentUris(SegmentGenerationJobSpec spec, List segmentUris, - List
headers, List parameters) - throws RetriableOperationException, AttemptsExceededException { - String tableName = spec.getTableSpec().getTableName(); - LOGGER.info("Start sending table {} segment URIs: {} to locations: {}", tableName, - Arrays.toString(segmentUris.subList(0, Math.min(5, segmentUris.size())).toArray()), - Arrays.toString(spec.getPinotClusterSpecs())); - for (String segmentUri : segmentUris) { - URI segmentURI = URI.create(segmentUri); - PinotFS outputDirFS = PinotFSFactory.create(segmentURI.getScheme()); - for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { - URI controllerURI; - try { - controllerURI = new URI(pinotClusterSpec.getControllerURI()); - } catch (URISyntaxException e) { - throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); - } - LOGGER.info("Sending table {} segment URI: {} to location: {} for ", tableName, segmentUri, controllerURI); - int attempts = 1; - if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushAttempts() > 0) { - attempts = spec.getPushJobSpec().getPushAttempts(); - } - long retryWaitMs = 1000L; - if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushRetryIntervalMillis() > 0) { - retryWaitMs = spec.getPushJobSpec().getPushRetryIntervalMillis(); - } - RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { - try { - SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT - .sendSegmentUri(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentUri, - headers, parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); - LOGGER.info("Response for pushing table {} segment uri {} to location {} - {}: {}", tableName, segmentUri, - controllerURI, response.getStatusCode(), response.getResponse()); - return true; - } catch (HttpErrorStatusException e) { - int statusCode = e.getStatusCode(); - if (statusCode >= 500) { - // Temporary exception - LOGGER.warn("Caught temporary exception while pushing table: {} segment uri: {} to {}, will retry", - tableName, segmentUri, controllerURI, e); - return false; - } else { - // Permanent exception - LOGGER.error("Caught permanent exception while pushing table: {} segment uri: {} to {}, won't retry", - tableName, segmentUri, controllerURI, e); - throw e; - } - } finally { - if (spec.isCleanUpOutputDir()) { - outputDirFS.delete(segmentURI, true); - } - } - }); - } - } - } - - /** - * This method takes a map of segment downloadURI to corresponding tar file path, and push those segments in - * metadata mode. - * The steps are: - * 1. Download segment from tar file path; - * 2. Untar segment metadata and creation meta files from the tar file to a segment metadata directory; - * 3. Tar this segment metadata directory into a tar file - * 4. Generate a POST request with segmentDownloadURI in header to push tar file to Pinot controller. - * - * @param spec is the segment generation job spec - * @param fileSystem is the PinotFs used to copy segment tar file - * @param segmentUriToTarPathMap contains the map of segment DownloadURI to segment tar file path - * @throws Exception - */ - public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, PinotFS fileSystem, - Map segmentUriToTarPathMap, List
headers, List parameters) - throws Exception { - String tableName = spec.getTableSpec().getTableName(); - LOGGER.info("Start pushing segment metadata: {} to locations: {} for table {}", segmentUriToTarPathMap, - Arrays.toString(spec.getPinotClusterSpecs()), tableName); - for (String segmentUriPath : segmentUriToTarPathMap.keySet()) { - String tarFilePath = segmentUriToTarPathMap.get(segmentUriPath); - String fileName = new File(tarFilePath).getName(); - // segments stored in Pinot deep store do not have .tar.gz extension - String segmentName = fileName.endsWith(Constants.TAR_GZ_FILE_EXT) - ? fileName.substring(0, fileName.length() - Constants.TAR_GZ_FILE_EXT.length()) : fileName; - SegmentNameUtils.validatePartialOrFullSegmentName(segmentName); - File segmentMetadataFile = generateSegmentMetadataFile(fileSystem, URI.create(tarFilePath)); - try { - for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { - URI controllerURI; - try { - controllerURI = new URI(pinotClusterSpec.getControllerURI()); - } catch (URISyntaxException e) { - throw new RuntimeException("Got invalid controller uri - '" + pinotClusterSpec.getControllerURI() + "'"); - } - LOGGER.info("Pushing segment: {} to location: {} for table {}", segmentName, controllerURI, tableName); - int attempts = 1; - if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushAttempts() > 0) { - attempts = spec.getPushJobSpec().getPushAttempts(); - } - long retryWaitMs = 1000L; - if (spec.getPushJobSpec() != null && spec.getPushJobSpec().getPushRetryIntervalMillis() > 0) { - retryWaitMs = spec.getPushJobSpec().getPushRetryIntervalMillis(); - } - RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { - try { - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentUriPath)); - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, - FileUploadDownloadClient.FileUploadType.METADATA.toString())); - if (spec.getPushJobSpec() != null) { - headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, - String.valueOf(spec.getPushJobSpec().getCopyToDeepStoreForMetadataPush()))); - } - - SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT - .uploadSegmentMetadata(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentName, - segmentMetadataFile, headers, parameters, - HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); - LOGGER.info("Response for pushing table {} segment {} to location {} - {}: {}", tableName, segmentName, - controllerURI, response.getStatusCode(), response.getResponse()); - return true; - } catch (HttpErrorStatusException e) { - int statusCode = e.getStatusCode(); - if (statusCode >= 500) { - // Temporary exception - LOGGER - .warn("Caught temporary exception while pushing table: {} segment: {} to {}, will retry", tableName, - segmentName, controllerURI, e); - return false; - } else { - // Permanent exception - LOGGER.error("Caught permanent exception while pushing table: {} segment: {} to {}, won't retry", - tableName, segmentName, controllerURI, e); - throw e; - } - } - }); - } - } finally { - FileUtils.deleteQuietly(segmentMetadataFile); - } - } - } - - public static Map getSegmentUriToTarPathMap(URI outputDirURI, PushJobSpec pushSpec, - String[] files) { - Map segmentUriToTarPathMap = new HashMap<>(); - PathMatcher pushFilePathMatcher = null; - if (pushSpec.getPushFileNamePattern() != null) { - pushFilePathMatcher = FileSystems.getDefault().getPathMatcher(pushSpec.getPushFileNamePattern()); - } - - for (String file : files) { - if (pushFilePathMatcher != null) { - if (!pushFilePathMatcher.matches(Paths.get(file))) { - continue; - } - } - - URI uri = URI.create(file); - if (uri.getPath().endsWith(Constants.TAR_GZ_FILE_EXT)) { - URI updatedURI = MinionPushUtils.generateSegmentTarURI(outputDirURI, uri, pushSpec.getSegmentUriPrefix(), - pushSpec.getSegmentUriSuffix()); - segmentUriToTarPathMap.put(updatedURI.toString(), file); - } - } - return segmentUriToTarPathMap; - } - - /** - * Generate a segment metadata only tar file, which contains only metadata.properties and creation.meta file. - * The purpose of this is to create a lean tar to push to Pinot controller for adding segments without downloading - * the complete segment and untar the segment tarball. - * - * 1. Download segment tar file to temp dir; - * 2. Extract only metadata.properties and creation.meta files from the segment tar file; - * 3. Tar both files into a segment metadata file. - * - */ - private static File generateSegmentMetadataFile(PinotFS fileSystem, URI tarFileURI) - throws Exception { - String uuid = UUID.randomUUID().toString(); - File tarFile = - new File(FileUtils.getTempDirectory(), "segmentTar-" + uuid + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION); - File segmentMetadataDir = new File(FileUtils.getTempDirectory(), "segmentMetadataDir-" + uuid); - try { - fileSystem.copyToLocalFile(tarFileURI, tarFile); - if (segmentMetadataDir.exists()) { - FileUtils.forceDelete(segmentMetadataDir); - } - FileUtils.forceMkdir(segmentMetadataDir); - - // Extract metadata.properties - LOGGER.info("Trying to untar Metadata file from: [{}] to [{}]", tarFile, segmentMetadataDir); - TarGzCompressionUtils.untarOneFile(tarFile, V1Constants.MetadataKeys.METADATA_FILE_NAME, - new File(segmentMetadataDir, V1Constants.MetadataKeys.METADATA_FILE_NAME)); - - // Extract creation.meta - LOGGER.info("Trying to untar CreationMeta file from: [{}] to [{}]", tarFile, segmentMetadataDir); - TarGzCompressionUtils.untarOneFile(tarFile, V1Constants.SEGMENT_CREATION_META, - new File(segmentMetadataDir, V1Constants.SEGMENT_CREATION_META)); - - File segmentMetadataTarFile = new File(FileUtils.getTempDirectory(), - "segmentMetadata-" + uuid + TarGzCompressionUtils.TAR_GZ_FILE_EXTENSION); - if (segmentMetadataTarFile.exists()) { - FileUtils.forceDelete(segmentMetadataTarFile); - } - LOGGER.info("Trying to tar segment metadata dir [{}] to [{}]", segmentMetadataDir, segmentMetadataTarFile); - TarGzCompressionUtils.createTarGzFile(segmentMetadataDir, segmentMetadataTarFile); - return segmentMetadataTarFile; - } finally { - FileUtils.deleteQuietly(tarFile); - FileUtils.deleteQuietly(segmentMetadataDir); - } - } -} diff --git a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/SegmentPushUtils.java b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/SegmentPushUtils.java index cee66b2b9af4..4f6423fadcfc 100644 --- a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/SegmentPushUtils.java +++ b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/SegmentPushUtils.java @@ -27,7 +27,6 @@ import java.nio.file.FileSystems; import java.nio.file.PathMatcher; import java.nio.file.Paths; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -36,6 +35,7 @@ import org.apache.commons.io.FileUtils; import org.apache.commons.lang3.StringUtils; import org.apache.http.Header; +import org.apache.http.NameValuePair; import org.apache.http.message.BasicHeader; import org.apache.pinot.common.auth.AuthProviderUtils; import org.apache.pinot.common.exception.HttpErrorStatusException; @@ -97,6 +97,49 @@ public static URI generateSegmentTarURI(URI dirURI, URI fileURI, String prefix, public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSystem, List tarFilePaths) throws RetriableOperationException, AttemptsExceededException { String tableName = spec.getTableSpec().getTableName(); + AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); + List
headers = AuthProviderUtils.toRequestHeaders(authProvider); + List parameters = FileUploadDownloadClient.makeTableParam(tableName); + pushSegments(spec, fileSystem, tarFilePaths, headers, parameters); + } + + public static void sendSegmentUris(SegmentGenerationJobSpec spec, List segmentUris) + throws RetriableOperationException, AttemptsExceededException { + String tableName = spec.getTableSpec().getTableName(); + AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); + List
headers = AuthProviderUtils.toRequestHeaders(authProvider); + List parameters = FileUploadDownloadClient.makeTableParam(tableName); + sendSegmentUris(spec, segmentUris, headers, parameters); + } + + /** + * This method takes a map of segment downloadURI to corresponding tar file path, and push those segments in + * metadata mode. + * The steps are: + * 1. Download segment from tar file path; + * 2. Untar segment metadata and creation meta files from the tar file to a segment metadata directory; + * 3. Tar this segment metadata directory into a tar file + * 4. Generate a POST request with segmentDownloadURI in header to push tar file to Pinot controller. + * + * @param spec is the segment generation job spec + * @param fileSystem is the PinotFs used to copy segment tar file + * @param segmentUriToTarPathMap contains the map of segment DownloadURI to segment tar file path + * @throws Exception + */ + public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, PinotFS fileSystem, + Map segmentUriToTarPathMap) + throws Exception { + String tableName = spec.getTableSpec().getTableName(); + AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); + List
headers = AuthProviderUtils.toRequestHeaders(authProvider); + List parameters = FileUploadDownloadClient.makeTableParam(tableName); + sendSegmentUriAndMetadata(spec, fileSystem, segmentUriToTarPathMap, headers, parameters); + } + + public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSystem, List tarFilePaths, + List
headers, List parameters) + throws RetriableOperationException, AttemptsExceededException { + String tableName = spec.getTableSpec().getTableName(); TableType tableType = tableName.endsWith("_" + TableType.REALTIME.name()) ? TableType.REALTIME : TableType.OFFLINE; boolean cleanUpOutputDir = spec.isCleanUpOutputDir(); LOGGER.info("Start pushing segments: {}... to locations: {} for table {}", @@ -108,7 +151,6 @@ public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSyste String fileName = tarFile.getName(); Preconditions.checkArgument(fileName.endsWith(Constants.TAR_GZ_FILE_EXT)); String segmentName = fileName.substring(0, fileName.length() - Constants.TAR_GZ_FILE_EXT.length()); - AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { URI controllerURI; try { @@ -129,8 +171,8 @@ public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSyste try (InputStream inputStream = fileSystem.open(tarFileURI)) { SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT.uploadSegment(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), - segmentName, inputStream, AuthProviderUtils.toRequestHeaders(authProvider), - FileUploadDownloadClient.makeTableParam(tableName), tableName, tableType); + segmentName, inputStream, headers, + parameters, tableName, tableType); LOGGER.info("Response for pushing table {} segment {} to location {} - {}: {}", tableName, segmentName, controllerURI, response.getStatusCode(), response.getResponse()); return true; @@ -157,7 +199,8 @@ public static void pushSegments(SegmentGenerationJobSpec spec, PinotFS fileSyste } } - public static void sendSegmentUris(SegmentGenerationJobSpec spec, List segmentUris) + public static void sendSegmentUris(SegmentGenerationJobSpec spec, List segmentUris, + List
headers, List parameters) throws RetriableOperationException, AttemptsExceededException { String tableName = spec.getTableSpec().getTableName(); LOGGER.info("Start sending table {} segment URIs: {} to locations: {}", tableName, @@ -166,7 +209,6 @@ public static void sendSegmentUris(SegmentGenerationJobSpec spec, List s for (String segmentUri : segmentUris) { URI segmentURI = URI.create(segmentUri); PinotFS outputDirFS = PinotFSFactory.create(segmentURI.getScheme()); - AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { URI controllerURI; try { @@ -187,8 +229,7 @@ public static void sendSegmentUris(SegmentGenerationJobSpec spec, List s try { SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT .sendSegmentUri(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentUri, - AuthProviderUtils.toRequestHeaders(authProvider), - FileUploadDownloadClient.makeTableParam(tableName), HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + headers, parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); LOGGER.info("Response for pushing table {} segment uri {} to location {} - {}: {}", tableName, segmentUri, controllerURI, response.getStatusCode(), response.getResponse()); return true; @@ -230,7 +271,7 @@ public static void sendSegmentUris(SegmentGenerationJobSpec spec, List s * @throws Exception */ public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, PinotFS fileSystem, - Map segmentUriToTarPathMap) + Map segmentUriToTarPathMap, List
headers, List parameters) throws Exception { String tableName = spec.getTableSpec().getTableName(); LOGGER.info("Start pushing segment metadata: {} to locations: {} for table {}", segmentUriToTarPathMap, @@ -243,7 +284,6 @@ public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, Pino ? fileName.substring(0, fileName.length() - Constants.TAR_GZ_FILE_EXT.length()) : fileName; SegmentNameUtils.validatePartialOrFullSegmentName(segmentName); File segmentMetadataFile = generateSegmentMetadataFile(fileSystem, URI.create(tarFilePath)); - AuthProvider authProvider = AuthProviderUtils.makeAuthProvider(spec.getAuthToken()); try { for (PinotClusterSpec pinotClusterSpec : spec.getPinotClusterSpecs()) { URI controllerURI; @@ -263,7 +303,6 @@ public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, Pino } RetryPolicies.exponentialBackoffRetryPolicy(attempts, retryWaitMs, 5).attempt(() -> { try { - List
headers = new ArrayList<>(); headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.DOWNLOAD_URI, segmentUriPath)); headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.UPLOAD_TYPE, FileUploadDownloadClient.FileUploadType.METADATA.toString())); @@ -271,12 +310,10 @@ public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, Pino headers.add(new BasicHeader(FileUploadDownloadClient.CustomHeaders.COPY_SEGMENT_TO_DEEP_STORE, String.valueOf(spec.getPushJobSpec().getCopyToDeepStoreForMetadataPush()))); } - headers.addAll(AuthProviderUtils.toRequestHeaders(authProvider)); - SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT - .uploadSegmentMetadata(FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentName, - segmentMetadataFile, headers, FileUploadDownloadClient.makeTableParam(tableName), - HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); + SimpleHttpResponse response = FILE_UPLOAD_DOWNLOAD_CLIENT.uploadSegmentMetadata( + FileUploadDownloadClient.getUploadSegmentURI(controllerURI), segmentName, + segmentMetadataFile, headers, parameters, HttpClient.DEFAULT_SOCKET_TIMEOUT_MS); LOGGER.info("Response for pushing table {} segment {} to location {} - {}: {}", tableName, segmentName, controllerURI, response.getStatusCode(), response.getResponse()); return true; @@ -284,9 +321,8 @@ public static void sendSegmentUriAndMetadata(SegmentGenerationJobSpec spec, Pino int statusCode = e.getStatusCode(); if (statusCode >= 500) { // Temporary exception - LOGGER - .warn("Caught temporary exception while pushing table: {} segment: {} to {}, will retry", tableName, - segmentName, controllerURI, e); + LOGGER.warn("Caught temporary exception while pushing table: {} segment: {} to {}, will retry", + tableName, segmentName, controllerURI, e); return false; } else { // Permanent exception From cced16bb7aeaa37f12ae7dce3102350667610a22 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 18 Nov 2022 11:36:29 +0530 Subject: [PATCH 07/16] Do not copy in case of tar push --- ...aseMultipleSegmentsConversionExecutor.java | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 507bf6f67cff..dd497b1cd006 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; @@ -256,9 +257,16 @@ public List executeTask(PinotTaskConfig pinotTaskConfig new BasicHeader(FileUploadDownloadClient.CustomHeaders.SEGMENT_ZK_METADATA_CUSTOM_MAP_MODIFIER, segmentZKMetadataCustomMapModifier.toJsonString()); - URI outputSegmentTarURI = moveSegmentToOutputPinotFS(pinotTaskConfig.getConfigs(), convertedTarredSegmentFile); - LOGGER.info("Moved generated segment from [{}] to location: [{}]", - convertedTarredSegmentFile, outputSegmentTarURI); + String pushMode = configs.get(BatchConfigProperties.PUSH_MODE); + URI outputSegmentTarURI; + if(BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase()) + != BatchConfigProperties.SegmentPushType.TAR) { + outputSegmentTarURI = moveSegmentToOutputPinotFS(configs, convertedTarredSegmentFile); + LOGGER.info("Moved generated segment from [{}] to location: [{}]", convertedTarredSegmentFile, + outputSegmentTarURI); + } else { + outputSegmentTarURI = convertedTarredSegmentFile.toURI(); + } List
httpHeaders = new ArrayList<>(); httpHeaders.add(segmentZKMetadataCustomMapModifierHeader); @@ -271,10 +279,7 @@ public List executeTask(PinotTaskConfig pinotTaskConfig TableNameBuilder.extractRawTableName(tableNameWithType)); List parameters = Arrays.asList(enableParallelPushProtectionParameter, tableNameParameter); - pushSegment(tableNameWithType, pinotTaskConfig.getConfigs(), outputSegmentTarURI, httpHeaders, parameters); -// SegmentConversionUtils -// .uploadSegment(configs, httpHeaders, parameters, tableNameWithType, resultSegmentName, uploadURL, -// convertedTarredSegmentFile); + pushSegment(tableNameWithType, configs, outputSegmentTarURI, httpHeaders, parameters); if (!FileUtils.deleteQuietly(convertedTarredSegmentFile)) { LOGGER.warn("Failed to delete tarred converted segment: {}", convertedTarredSegmentFile.getAbsolutePath()); } From c691c0c008a734c5c8be067c0f0bed2fb9c834ee Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 18 Nov 2022 11:42:38 +0530 Subject: [PATCH 08/16] Remove duplicate code for filesystem --- ...aseMultipleSegmentsConversionExecutor.java | 7 +- .../{TaskUtils.java => MinionTaskUtils.java} | 10 +-- .../SegmentGenerationAndPushTaskExecutor.java | 9 ++- ...SegmentGenerationAndPushTaskGenerator.java | 3 +- .../SegmentGenerationAndPushTaskUtils.java | 73 ------------------- 5 files changed, 15 insertions(+), 87 deletions(-) rename pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/{TaskUtils.java => MinionTaskUtils.java} (90%) delete mode 100644 pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskUtils.java diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index dd497b1cd006..0dbb039bdd98 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -26,7 +26,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; @@ -318,10 +317,10 @@ private void pushSegment(String tableName, Map taskConfigs, URI if (taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); } - try (PinotFS outputFileFS = TaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { + try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { case TAR: - try (PinotFS pinotFS = TaskUtils.getLocalPinotFs()) { + try (PinotFS pinotFS = MinionTaskUtils.getLocalPinotFs()) { SegmentPushUtils.pushSegments( spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString()), headers, parameters); } catch (RetriableOperationException | AttemptsExceededException e) { @@ -380,7 +379,7 @@ private URI moveSegmentToOutputPinotFS(Map taskConfigs, File loc return localSegmentTarFile.toURI(); } URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); - try (PinotFS outputFileFS = TaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { + try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { URI outputSegmentTarURI = URI.create(outputSegmentDirURI + localSegmentTarFile.getName()); if (!Boolean.parseBoolean(taskConfigs.get(BatchConfigProperties.OVERWRITE_OUTPUT)) && outputFileFS.exists( outputSegmentDirURI)) { diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java similarity index 90% rename from pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java rename to pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java index 538fdbb13b8f..22c163f4b512 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/TaskUtils.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java @@ -29,11 +29,11 @@ import org.apache.pinot.spi.utils.IngestionConfigUtils; -public class TaskUtils { - private TaskUtils() { +public class MinionTaskUtils { + private MinionTaskUtils() { } - static PinotFS getInputPinotFS(Map taskConfigs, URI fileURI) + public static PinotFS getInputPinotFS(Map taskConfigs, URI fileURI) throws Exception { String fileURIScheme = fileURI.getScheme(); if (fileURIScheme == null) { @@ -50,7 +50,7 @@ static PinotFS getInputPinotFS(Map taskConfigs, URI fileURI) return PinotFSFactory.create(fileURIScheme); } - static PinotFS getOutputPinotFS(Map taskConfigs, URI fileURI) + public static PinotFS getOutputPinotFS(Map taskConfigs, URI fileURI) throws Exception { String fileURIScheme = (fileURI == null) ? null : fileURI.getScheme(); if (fileURIScheme == null) { @@ -67,7 +67,7 @@ static PinotFS getOutputPinotFS(Map taskConfigs, URI fileURI) return PinotFSFactory.create(fileURIScheme); } - static PinotFS getLocalPinotFs() { + public static PinotFS getLocalPinotFs() { return new LocalPinotFS(); } } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java index a7d5127c0928..03ce7807ab5b 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java @@ -37,6 +37,7 @@ import org.apache.pinot.plugin.ingestion.batch.common.SegmentGenerationTaskRunner; import org.apache.pinot.plugin.minion.tasks.BaseTaskExecutor; import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; import org.apache.pinot.segment.local.utils.SegmentPushUtils; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; @@ -176,10 +177,10 @@ private void pushSegment(String tableName, Map taskConfigs, URI if (taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); } - try (PinotFS outputFileFS = SegmentGenerationAndPushTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { + try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { case TAR: - try (PinotFS pinotFS = SegmentGenerationAndPushTaskUtils.getLocalPinotFs()) { + try (PinotFS pinotFS = MinionTaskUtils.getLocalPinotFs()) { SegmentPushUtils.pushSegments(spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString())); } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); @@ -237,7 +238,7 @@ private URI moveSegmentToOutputPinotFS(Map taskConfigs, File loc return localSegmentTarFile.toURI(); } URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); - try (PinotFS outputFileFS = SegmentGenerationAndPushTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { + try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { URI outputSegmentTarURI = URI.create(outputSegmentDirURI + localSegmentTarFile.getName()); if (!Boolean.parseBoolean(taskConfigs.get(BatchConfigProperties.OVERWRITE_OUTPUT)) && outputFileFS.exists( outputSegmentDirURI)) { @@ -269,7 +270,7 @@ protected SegmentGenerationTaskSpec generateTaskSpec(Map taskCon SegmentGenerationTaskSpec taskSpec = new SegmentGenerationTaskSpec(); URI inputFileURI = URI.create(taskConfigs.get(BatchConfigProperties.INPUT_DATA_FILE_URI_KEY)); - try (PinotFS inputFileFS = SegmentGenerationAndPushTaskUtils.getInputPinotFS(taskConfigs, inputFileURI)) { + try (PinotFS inputFileFS = MinionTaskUtils.getInputPinotFS(taskConfigs, inputFileURI)) { File localInputTempDir = new File(localTempDir, "input"); FileUtils.forceMkdir(localInputTempDir); File localOutputTempDir = new File(localTempDir, "output"); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskGenerator.java index eff429ec4aaf..e1bdd97cac15 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskGenerator.java @@ -41,6 +41,7 @@ import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; import org.apache.pinot.core.common.MinionConstants; import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; import org.apache.pinot.spi.annotations.minion.TaskGenerator; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableTaskConfig; @@ -323,7 +324,7 @@ private void updateRecordReaderConfigs(Map batchConfigMap) { private List getInputFilesFromDirectory(Map batchConfigMap, URI inputDirURI, Set existingSegmentInputFileURIs) throws Exception { - try (PinotFS inputDirFS = SegmentGenerationAndPushTaskUtils.getInputPinotFS(batchConfigMap, inputDirURI)) { + try (PinotFS inputDirFS = MinionTaskUtils.getInputPinotFS(batchConfigMap, inputDirURI)) { String includeFileNamePattern = batchConfigMap.get(BatchConfigProperties.INCLUDE_FILE_NAME_PATTERN); String excludeFileNamePattern = batchConfigMap.get(BatchConfigProperties.EXCLUDE_FILE_NAME_PATTERN); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskUtils.java deleted file mode 100644 index 913184943fe3..000000000000 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskUtils.java +++ /dev/null @@ -1,73 +0,0 @@ -/** - * 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.minion.tasks.segmentgenerationandpush; - -import java.net.URI; -import java.util.Map; -import org.apache.pinot.spi.env.PinotConfiguration; -import org.apache.pinot.spi.filesystem.LocalPinotFS; -import org.apache.pinot.spi.filesystem.PinotFS; -import org.apache.pinot.spi.filesystem.PinotFSFactory; -import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; -import org.apache.pinot.spi.plugin.PluginManager; -import org.apache.pinot.spi.utils.IngestionConfigUtils; - - -public class SegmentGenerationAndPushTaskUtils { - private SegmentGenerationAndPushTaskUtils() { - } - - static PinotFS getInputPinotFS(Map taskConfigs, URI fileURI) - throws Exception { - String fileURIScheme = fileURI.getScheme(); - if (fileURIScheme == null) { - return new LocalPinotFS(); - } - // Try to create PinotFS using given Input FileSystem config always - String fsClass = taskConfigs.get(BatchConfigProperties.INPUT_FS_CLASS); - if (fsClass != null) { - PinotFS pinotFS = PluginManager.get().createInstance(fsClass); - PinotConfiguration fsProps = IngestionConfigUtils.getInputFsProps(taskConfigs); - pinotFS.init(fsProps); - return pinotFS; - } - return PinotFSFactory.create(fileURIScheme); - } - - static PinotFS getOutputPinotFS(Map taskConfigs, URI fileURI) - throws Exception { - String fileURIScheme = (fileURI == null) ? null : fileURI.getScheme(); - if (fileURIScheme == null) { - return new LocalPinotFS(); - } - // Try to create PinotFS using given Input FileSystem config always - String fsClass = taskConfigs.get(BatchConfigProperties.OUTPUT_FS_CLASS); - if (fsClass != null) { - PinotFS pinotFS = PluginManager.get().createInstance(fsClass); - PinotConfiguration fsProps = IngestionConfigUtils.getOutputFsProps(taskConfigs); - pinotFS.init(fsProps); - return pinotFS; - } - return PinotFSFactory.create(fileURIScheme); - } - - static PinotFS getLocalPinotFs() { - return new LocalPinotFS(); - } -} From b868117cb88cb98632561ea7c85f4850a2bf250f Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 18 Nov 2022 14:36:47 +0530 Subject: [PATCH 09/16] Add metadata push to realtime to offline task --- ...eSegmentsMinionClusterIntegrationTest.java | 7 +++- ...aseMultipleSegmentsConversionExecutor.java | 4 +- .../mergerollup/MergeRollupTaskGenerator.java | 5 ++- ...ealtimeToOfflineSegmentsTaskGenerator.java | 42 ++++++++++++++++++- .../SegmentGenerationAndPushTaskExecutor.java | 2 +- 5 files changed, 53 insertions(+), 7 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java index a8eb0a4c4065..3f7f762ce7cb 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java @@ -45,6 +45,7 @@ import org.apache.pinot.spi.data.DateTimeFieldSpec; import org.apache.pinot.spi.data.FieldSpec.DataType; import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; @@ -113,8 +114,12 @@ public void setUp() new TimestampConfig(Arrays.asList(TimestampIndexGranularity.HOUR, TimestampIndexGranularity.DAY, TimestampIndexGranularity.WEEK, TimestampIndexGranularity.MONTH)), null); realtimeTableConfig.setFieldConfigList(Collections.singletonList(tsFieldConfig)); + + Map taskConfigs = new HashMap<>(); + taskConfigs.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); + realtimeTableConfig.setTaskConfig(new TableTaskConfig( - Collections.singletonMap(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, new HashMap<>()))); + Collections.singletonMap(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, taskConfigs))); addTableConfig(realtimeTableConfig); TableConfig offlineTableConfig = createOfflineTableConfig(); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 0dbb039bdd98..0ae2f194fbca 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -258,7 +258,7 @@ public List executeTask(PinotTaskConfig pinotTaskConfig String pushMode = configs.get(BatchConfigProperties.PUSH_MODE); URI outputSegmentTarURI; - if(BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase()) + if (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase()) != BatchConfigProperties.SegmentPushType.TAR) { outputSegmentTarURI = moveSegmentToOutputPinotFS(configs, convertedTarredSegmentFile); LOGGER.info("Moved generated segment from [{}] to location: [{}]", convertedTarredSegmentFile, @@ -278,7 +278,7 @@ public List executeTask(PinotTaskConfig pinotTaskConfig TableNameBuilder.extractRawTableName(tableNameWithType)); List parameters = Arrays.asList(enableParallelPushProtectionParameter, tableNameParameter); - pushSegment(tableNameWithType, configs, outputSegmentTarURI, httpHeaders, parameters); + pushSegment(tableNameParameter.getValue(), configs, outputSegmentTarURI, httpHeaders, parameters); if (!FileUtils.deleteQuietly(convertedTarredSegmentFile)) { LOGGER.warn("Failed to delete tarred converted segment: {}", convertedTarredSegmentFile.getAbsolutePath()); } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java index 8cdade2de8d7..f8ea78c0b72a 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java @@ -560,7 +560,8 @@ private List createPinotTaskConfigs(List sel } } - configs.put(BatchConfigProperties.OVERWRITE_OUTPUT, taskConfigs.get(BatchConfigProperties.OVERWRITE_OUTPUT)); + configs.put(BatchConfigProperties.OVERWRITE_OUTPUT, + taskConfigs.getOrDefault(BatchConfigProperties.OVERWRITE_OUTPUT, "false")); configs.put(MergeRollupTask.MERGE_TYPE_KEY, mergeConfigs.get(MergeTask.MERGE_TYPE_KEY)); configs.put(MergeRollupTask.MERGE_LEVEL_KEY, mergeLevel); configs.put(MergeTask.PARTITION_BUCKET_TIME_PERIOD_KEY, mergeConfigs.get(MergeTask.BUCKET_TIME_PERIOD_KEY)); @@ -579,7 +580,7 @@ private List createPinotTaskConfigs(List sel private Map getPushTaskConfig(Map batchConfigMap, String downloadUrls) { try { - String[] downloadURLList = downloadUrls.split(MinionConstants.SEGMENT_NAME_SEPARATOR); + String[] downloadURLList = downloadUrls.split(MinionConstants.URL_SEPARATOR); if (downloadURLList.length > 0) { String downloadUrl = downloadURLList[0]; URI downloadURI = URI.create(downloadUrl); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java index f718821082d0..861d53250778 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java @@ -19,6 +19,7 @@ package org.apache.pinot.plugin.minion.tasks.realtimetoofflinesegments; import com.google.common.base.Preconditions; +import java.net.URI; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -30,6 +31,7 @@ import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.minion.RealtimeToOfflineSegmentsTaskMetadata; +import org.apache.pinot.common.segment.generation.SegmentGenerationUtils; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator; @@ -41,6 +43,7 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableTaskConfig; import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.utils.CommonConstants.Segment; import org.apache.pinot.spi.utils.IngestionConfigUtils; @@ -82,6 +85,8 @@ public class RealtimeToOfflineSegmentsTaskGenerator extends BaseTaskGenerator { private static final String DEFAULT_BUCKET_PERIOD = "1d"; private static final String DEFAULT_BUFFER_PERIOD = "2d"; + private static final BatchConfigProperties.SegmentPushType DEFAULT_SEGMENT_PUSH_TYPE = + BatchConfigProperties.SegmentPushType.TAR; @Override public String getTaskType() { @@ -204,7 +209,8 @@ public List generateTasks(List tableConfigs) { continue; } - Map configs = new HashMap<>(); + Map configs = getPushTaskConfig(taskConfigs, + StringUtils.join(downloadURLs, MinionConstants.URL_SEPARATOR)); configs.put(MinionConstants.TABLE_NAME_KEY, realtimeTableName); configs.put(MinionConstants.SEGMENT_NAME_KEY, StringUtils.join(segmentNames, ",")); configs.put(MinionConstants.DOWNLOAD_URL_KEY, StringUtils.join(downloadURLs, MinionConstants.URL_SEPARATOR)); @@ -321,4 +327,38 @@ private long getWatermarkMs(String realtimeTableName, List co } return realtimeToOfflineSegmentsTaskMetadata.getWatermarkMs(); } + + private Map getPushTaskConfig(Map batchConfigMap, String downloadUrls) { + try { + String[] downloadURLList = downloadUrls.split(MinionConstants.URL_SEPARATOR); + if (downloadURLList.length > 0) { + String downloadUrl = downloadURLList[0]; + URI downloadURI = URI.create(downloadUrl); + URI outputDirURI = null; + if (!downloadURI.getScheme().contentEquals("http")) { + String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); + outputDirURI = URI.create(outputDir); + } + String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); + + Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); + if (outputDirURI != null) { + URI outputSegmentDirURI = SegmentGenerationUtils.getRelativeOutputPath( + outputDirURI, downloadURI, outputDirURI); + singleFileGenerationTaskConfig.put( + BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); + } + if ((outputDirURI == null) || (pushMode == null)) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); + } else { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + } + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl()); + return singleFileGenerationTaskConfig; + } + return batchConfigMap; + } catch (Exception e) { + return batchConfigMap; + } + } } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java index 03ce7807ab5b..6475d6f895a6 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/segmentgenerationandpush/SegmentGenerationAndPushTaskExecutor.java @@ -36,8 +36,8 @@ import org.apache.pinot.minion.event.MinionEventObservers; import org.apache.pinot.plugin.ingestion.batch.common.SegmentGenerationTaskRunner; import org.apache.pinot.plugin.minion.tasks.BaseTaskExecutor; -import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; +import org.apache.pinot.plugin.minion.tasks.SegmentConversionResult; import org.apache.pinot.segment.local.utils.SegmentPushUtils; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.data.Schema; From 169332f8fe3b4c438351d28eb48bebbf066d483d Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 18 Nov 2022 17:05:22 +0530 Subject: [PATCH 10/16] Use data dir as output dir --- ...RealtimeToOfflineSegmentsTaskGenerator.java | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java index 861d53250778..48a3d0209b70 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java @@ -209,7 +209,7 @@ public List generateTasks(List tableConfigs) { continue; } - Map configs = getPushTaskConfig(taskConfigs, + Map configs = getPushTaskConfig(realtimeTableName, taskConfigs, StringUtils.join(downloadURLs, MinionConstants.URL_SEPARATOR)); configs.put(MinionConstants.TABLE_NAME_KEY, realtimeTableName); configs.put(MinionConstants.SEGMENT_NAME_KEY, StringUtils.join(segmentNames, ",")); @@ -328,7 +328,7 @@ private long getWatermarkMs(String realtimeTableName, List co return realtimeToOfflineSegmentsTaskMetadata.getWatermarkMs(); } - private Map getPushTaskConfig(Map batchConfigMap, String downloadUrls) { + private Map getPushTaskConfig(String tableName, Map batchConfigMap, String downloadUrls) { try { String[] downloadURLList = downloadUrls.split(MinionConstants.URL_SEPARATOR); if (downloadURLList.length > 0) { @@ -337,18 +337,15 @@ private Map getPushTaskConfig(Map batchConfigMap URI outputDirURI = null; if (!downloadURI.getScheme().contentEquals("http")) { String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); - outputDirURI = URI.create(outputDir); + outputDirURI = SegmentGenerationUtils.getDirectoryURI(outputDir); + } else { + outputDirURI = SegmentGenerationUtils.getDirectoryURI(_clusterInfoAccessor.getDataDir() + "/" + tableName); } String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); - if (outputDirURI != null) { - URI outputSegmentDirURI = SegmentGenerationUtils.getRelativeOutputPath( - outputDirURI, downloadURI, outputDirURI); - singleFileGenerationTaskConfig.put( - BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); - } - if ((outputDirURI == null) || (pushMode == null)) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputDirURI.toString()); + if (pushMode == null) { singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); } else { singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); @@ -358,6 +355,7 @@ private Map getPushTaskConfig(Map batchConfigMap } return batchConfigMap; } catch (Exception e) { + LOGGER.warn("Error occurred while generating push task config", e); return batchConfigMap; } } From 198c2967e1a7f8afca001a31b3becce937ebbe1b Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 18 Nov 2022 18:42:50 +0530 Subject: [PATCH 11/16] Add tests for realtime to offline metadata push --- .../tests/BaseClusterIntegrationTest.java | 6 +- ...eSegmentsMinionClusterIntegrationTest.java | 128 +++++++++++++++++- ...ealtimeToOfflineSegmentsTaskGenerator.java | 3 +- 3 files changed, 129 insertions(+), 8 deletions(-) diff --git a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java index b133a275514b..11ffabe953ae 100644 --- a/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java +++ b/pinot-integration-test-base/src/test/java/org/apache/pinot/integration/tests/BaseClusterIntegrationTest.java @@ -607,13 +607,17 @@ protected void waitForAllDocsLoaded(long timeoutMs) } protected void waitForDocsLoaded(long timeoutMs, boolean raiseError) { + waitForDocsLoaded(timeoutMs, raiseError, getTableName()); + } + + protected void waitForDocsLoaded(long timeoutMs, boolean raiseError, String tableName) { final long countStarResult = getCountStarResult(); TestUtils.waitForCondition(new Function() { @Nullable @Override public Boolean apply(@Nullable Void aVoid) { try { - return getCurrentCountStarResult() == countStarResult; + return getCurrentCountStarResult(tableName) == countStarResult; } catch (Exception e) { return null; } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java index 3f7f762ce7cb..852230188b31 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/RealtimeToOfflineSegmentsMinionClusterIntegrationTest.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.commons.io.FileUtils; import org.apache.helix.task.TaskState; import org.apache.helix.zookeeper.datamodel.ZNRecord; @@ -38,6 +39,7 @@ import org.apache.pinot.spi.config.table.SegmentPartitionConfig; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableTaskConfig; +import org.apache.pinot.spi.config.table.TableType; import org.apache.pinot.spi.config.table.TimestampConfig; import org.apache.pinot.spi.config.table.TimestampIndexGranularity; import org.apache.pinot.spi.config.table.ingestion.IngestionConfig; @@ -47,6 +49,7 @@ import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.spi.utils.builder.TableNameBuilder; import org.apache.pinot.util.TestUtils; import org.testng.annotations.AfterClass; @@ -69,7 +72,11 @@ public class RealtimeToOfflineSegmentsMinionClusterIntegrationTest extends BaseC private PinotTaskManager _taskManager; private String _realtimeTableName; private String _offlineTableName; + + private String _realtimeMetadataTableName; + private String _offlineMetadataTableName; private long _dataSmallestTimeMs; + private long _dataSmallestMetadataTableTimeMs; @Override protected SegmentPartitionConfig getSegmentPartitionConfig() { @@ -117,7 +124,6 @@ public void setUp() Map taskConfigs = new HashMap<>(); taskConfigs.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); - realtimeTableConfig.setTaskConfig(new TableTaskConfig( Collections.singletonMap(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, taskConfigs))); addTableConfig(realtimeTableConfig); @@ -126,6 +132,24 @@ public void setUp() offlineTableConfig.setFieldConfigList(Collections.singletonList(tsFieldConfig)); addTableConfig(offlineTableConfig); + Map taskConfigsWithMetadata = new HashMap<>(); + taskConfigsWithMetadata.put(BatchConfigProperties.OVERWRITE_OUTPUT, "true"); + taskConfigsWithMetadata.put( + BatchConfigProperties.PUSH_MODE, BatchConfigProperties.SegmentPushType.METADATA.toString()); + String tableWithMetadataPush = "myTable2"; + TableConfig realtimeMetadataTableConfig = createRealtimeTableConfig(avroFiles.get(0), tableWithMetadataPush, + new TableTaskConfig(Collections.singletonMap( + MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, taskConfigsWithMetadata))); + realtimeMetadataTableConfig.setIngestionConfig(ingestionConfig); + realtimeMetadataTableConfig.setFieldConfigList(Collections.singletonList(tsFieldConfig)); + addTableConfig(realtimeMetadataTableConfig); + + TableConfig offlineMetadataTableConfig = + createOfflineTableConfig(tableWithMetadataPush, null, getSegmentPartitionConfig()); + offlineMetadataTableConfig.setFieldConfigList(Collections.singletonList(tsFieldConfig)); + addTableConfig(offlineMetadataTableConfig); + + // Push data into Kafka pushAvroIntoKafka(avroFiles); @@ -135,11 +159,17 @@ public void setUp() // Wait for all documents loaded waitForAllDocsLoaded(600_000L); + waitForDocsLoaded(600_000L, true, tableWithMetadataPush); + + _taskResourceManager = _controllerStarter.getHelixTaskResourceManager(); _taskManager = _controllerStarter.getTaskManager(); _realtimeTableName = TableNameBuilder.REALTIME.tableNameWithType(getTableName()); _offlineTableName = TableNameBuilder.OFFLINE.tableNameWithType(getTableName()); + _realtimeMetadataTableName = TableNameBuilder.REALTIME.tableNameWithType(tableWithMetadataPush); + _offlineMetadataTableName = TableNameBuilder.OFFLINE.tableNameWithType(tableWithMetadataPush); + List segmentsZKMetadata = _helixResourceManager.getSegmentsZKMetadata(_realtimeTableName); long minSegmentTimeMs = Long.MAX_VALUE; for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { @@ -148,8 +178,42 @@ public void setUp() } } _dataSmallestTimeMs = minSegmentTimeMs; + + segmentsZKMetadata = _helixResourceManager.getSegmentsZKMetadata(_realtimeMetadataTableName); + minSegmentTimeMs = Long.MAX_VALUE; + for (SegmentZKMetadata segmentZKMetadata : segmentsZKMetadata) { + if (segmentZKMetadata.getStatus() == CommonConstants.Segment.Realtime.Status.DONE) { + minSegmentTimeMs = Math.min(minSegmentTimeMs, segmentZKMetadata.getStartTimeMs()); + } + } + _dataSmallestMetadataTableTimeMs = minSegmentTimeMs; + } + + private TableConfig createOfflineTableConfig(String tableName, @Nullable TableTaskConfig taskConfig, + @Nullable SegmentPartitionConfig partitionConfig) { + return new TableConfigBuilder(TableType.OFFLINE).setTableName(tableName).setSchemaName(getSchemaName()) + .setTimeColumnName(getTimeColumnName()).setSortedColumn(getSortedColumn()) + .setInvertedIndexColumns(getInvertedIndexColumns()).setNoDictionaryColumns(getNoDictionaryColumns()) + .setRangeIndexColumns(getRangeIndexColumns()).setBloomFilterColumns(getBloomFilterColumns()) + .setFieldConfigList(getFieldConfigs()).setNumReplicas(getNumReplicas()).setSegmentVersion(getSegmentVersion()) + .setLoadMode(getLoadMode()).setTaskConfig(taskConfig).setBrokerTenant(getBrokerTenant()) + .setServerTenant(getServerTenant()).setIngestionConfig(getIngestionConfig()) + .setNullHandlingEnabled(getNullHandlingEnabled()).setSegmentPartitionConfig(partitionConfig).build(); + } + + protected TableConfig createRealtimeTableConfig(File sampleAvroFile, String tableName, TableTaskConfig taskConfig) { + AvroFileSchemaKafkaAvroMessageDecoder._avroFile = sampleAvroFile; + return new TableConfigBuilder(TableType.REALTIME).setTableName(tableName).setSchemaName(getSchemaName()) + .setTimeColumnName(getTimeColumnName()).setSortedColumn(getSortedColumn()) + .setInvertedIndexColumns(getInvertedIndexColumns()).setNoDictionaryColumns(getNoDictionaryColumns()) + .setRangeIndexColumns(getRangeIndexColumns()).setBloomFilterColumns(getBloomFilterColumns()) + .setFieldConfigList(getFieldConfigs()).setNumReplicas(getNumReplicas()).setSegmentVersion(getSegmentVersion()) + .setLoadMode(getLoadMode()).setTaskConfig(taskConfig).setBrokerTenant(getBrokerTenant()) + .setServerTenant(getServerTenant()).setIngestionConfig(getIngestionConfig()).setQueryConfig(getQueryconfig()) + .setLLC(useLlc()).setStreamConfigs(getStreamConfigs()).setNullHandlingEnabled(getNullHandlingEnabled()).build(); } + @Test public void testRealtimeToOfflineSegmentsTask() throws Exception { @@ -168,14 +232,16 @@ public void testRealtimeToOfflineSegmentsTask() long expectedWatermark = _dataSmallestTimeMs + 86400000; for (int i = 0; i < 3; i++) { // Schedule task - assertNotNull(_taskManager.scheduleTasks().get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); + assertNotNull(_taskManager.scheduleTasks(_realtimeTableName) + .get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); assertTrue(_taskResourceManager.getTaskQueues().contains( PinotHelixTaskResourceManager.getHelixJobQueueName(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE))); // Should not generate more tasks - assertNull(_taskManager.scheduleTasks().get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); + assertNull(_taskManager.scheduleTasks(_realtimeTableName) + .get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); // Wait at most 600 seconds for all tasks COMPLETED - waitForTaskToComplete(expectedWatermark); + waitForTaskToComplete(expectedWatermark, _realtimeTableName); // check segment is in offline segmentsZKMetadata = _helixResourceManager.getSegmentsZKMetadata(_offlineTableName); assertEquals(segmentsZKMetadata.size(), (numOfflineSegmentsPerTask * (i + 1))); @@ -199,7 +265,57 @@ public void testRealtimeToOfflineSegmentsTask() testHardcodedQueries(); } - private void waitForTaskToComplete(long expectedWatermark) { + @Test + public void testRealtimeToOfflineSegmentsMetadataPushTask() + throws Exception { + List segmentsZKMetadata = _helixResourceManager.getSegmentsZKMetadata(_offlineMetadataTableName); + assertTrue(segmentsZKMetadata.isEmpty()); + + // The number of offline segments would be equal to the product of number of partitions for all the + // partition columns if segment partitioning is configured. + SegmentPartitionConfig segmentPartitionConfig = + getOfflineTableConfig().getIndexingConfig().getSegmentPartitionConfig(); + int numOfflineSegmentsPerTask = + segmentPartitionConfig != null ? segmentPartitionConfig.getColumnPartitionMap().values().stream() + .map(ColumnPartitionConfig::getNumPartitions).reduce((a, b) -> a * b) + .orElseThrow(() -> new RuntimeException("Expected accumulated result but not found.")) : 1; + + long expectedWatermark = _dataSmallestMetadataTableTimeMs + 86400000; + _taskManager.cleanUpTask(); + for (int i = 0; i < 3; i++) { + // Schedule task + assertNotNull(_taskManager.scheduleTasks(_realtimeMetadataTableName) + .get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); + assertTrue(_taskResourceManager.getTaskQueues().contains( + PinotHelixTaskResourceManager.getHelixJobQueueName(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE))); + // Should not generate more tasks + assertNull(_taskManager.scheduleTasks(_realtimeMetadataTableName) + .get(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE)); + + // Wait at most 600 seconds for all tasks COMPLETED + waitForTaskToComplete(expectedWatermark, _realtimeMetadataTableName); + // check segment is in offline + segmentsZKMetadata = _helixResourceManager.getSegmentsZKMetadata(_offlineMetadataTableName); + assertEquals(segmentsZKMetadata.size(), (numOfflineSegmentsPerTask * (i + 1))); + + long expectedOfflineSegmentTimeMs = expectedWatermark - 86400000; + for (int j = (numOfflineSegmentsPerTask * i); j < segmentsZKMetadata.size(); j++) { + SegmentZKMetadata segmentZKMetadata = segmentsZKMetadata.get(j); + assertEquals(segmentZKMetadata.getStartTimeMs(), expectedOfflineSegmentTimeMs); + assertEquals(segmentZKMetadata.getEndTimeMs(), expectedOfflineSegmentTimeMs); + if (segmentPartitionConfig != null) { + assertEquals(segmentZKMetadata.getPartitionMetadata().getColumnPartitionMap().keySet(), + segmentPartitionConfig.getColumnPartitionMap().keySet()); + for (String partitionColumn : segmentPartitionConfig.getColumnPartitionMap().keySet()) { + assertEquals(segmentZKMetadata.getPartitionMetadata().getPartitions(partitionColumn).size(), 1); + } + } + } + expectedWatermark += 86400000; + } + } + + private void waitForTaskToComplete(long expectedWatermark, String realtimeTableName) { TestUtils.waitForCondition(input -> { // Check task state for (TaskState taskState : _taskResourceManager.getTaskStates( @@ -213,7 +329,7 @@ private void waitForTaskToComplete(long expectedWatermark) { // Check segment ZK metadata ZNRecord znRecord = _taskManager.getClusterInfoAccessor() - .getMinionTaskMetadataZNRecord(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, _realtimeTableName); + .getMinionTaskMetadataZNRecord(MinionConstants.RealtimeToOfflineSegmentsTask.TASK_TYPE, realtimeTableName); RealtimeToOfflineSegmentsTaskMetadata minionTaskMetadata = znRecord != null ? RealtimeToOfflineSegmentsTaskMetadata.fromZNRecord(znRecord) : null; assertNotNull(minionTaskMetadata); diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java index 48a3d0209b70..470856c995c7 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java @@ -328,7 +328,8 @@ private long getWatermarkMs(String realtimeTableName, List co return realtimeToOfflineSegmentsTaskMetadata.getWatermarkMs(); } - private Map getPushTaskConfig(String tableName, Map batchConfigMap, String downloadUrls) { + private Map getPushTaskConfig(String tableName, + Map batchConfigMap, String downloadUrls) { try { String[] downloadURLList = downloadUrls.split(MinionConstants.URL_SEPARATOR); if (downloadURLList.length > 0) { From 4486ae8e0b53c375a0d2ca3cad75cc4925af1ae3 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Mon, 21 Nov 2022 16:27:36 +0530 Subject: [PATCH 12/16] use controller data dir for output --- .../plugin/minion/tasks/MinionTaskUtils.java | 33 +++++++++++++++ .../mergerollup/MergeRollupTaskGenerator.java | 42 ++----------------- ...ealtimeToOfflineSegmentsTaskGenerator.java | 40 ++---------------- 3 files changed, 39 insertions(+), 76 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java index 22c163f4b512..2ed6e4b1d17c 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java @@ -19,7 +19,9 @@ package org.apache.pinot.plugin.minion.tasks; import java.net.URI; +import java.util.HashMap; import java.util.Map; +import org.apache.pinot.controller.helix.core.minion.ClusterInfoAccessor; import org.apache.pinot.spi.env.PinotConfiguration; import org.apache.pinot.spi.filesystem.LocalPinotFS; import org.apache.pinot.spi.filesystem.PinotFS; @@ -30,6 +32,9 @@ public class MinionTaskUtils { + private static final BatchConfigProperties.SegmentPushType DEFAULT_SEGMENT_PUSH_TYPE = + BatchConfigProperties.SegmentPushType.TAR; + private MinionTaskUtils() { } @@ -67,6 +72,34 @@ public static PinotFS getOutputPinotFS(Map taskConfigs, URI file return PinotFSFactory.create(fileURIScheme); } + public static Map getPushTaskConfig(String tableName, Map taskConfigs, + ClusterInfoAccessor clusterInfoAccessor) { + try { + URI outputDirURI = URI.create(clusterInfoAccessor.getDataDir() + "/" + tableName); + String outputDirURIScheme = outputDirURI.getScheme(); + String pushMode = IngestionConfigUtils.getPushMode(taskConfigs); + + Map singleFileGenerationTaskConfig = new HashMap<>(taskConfigs); + if (!isLocalOutputDir(outputDirURIScheme)) { + singleFileGenerationTaskConfig.put( + BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputDirURI.toString()); + } + if (isLocalOutputDir(outputDirURIScheme) || (pushMode == null)) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); + } else { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + } + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, clusterInfoAccessor.getVipUrl()); + return singleFileGenerationTaskConfig; + } catch (Exception e) { + return taskConfigs; + } + } + + public static boolean isLocalOutputDir(String outputDirURIScheme) { + return outputDirURIScheme == null || outputDirURIScheme.startsWith("file"); + } + public static PinotFS getLocalPinotFs() { return new LocalPinotFS(); } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java index f8ea78c0b72a..aa8dd86cc410 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/mergerollup/MergeRollupTaskGenerator.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.plugin.minion.tasks.mergerollup; -import java.net.URI; import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; @@ -39,7 +38,6 @@ import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.metrics.ControllerMetrics; import org.apache.pinot.common.minion.MergeRollupTaskMetadata; -import org.apache.pinot.common.segment.generation.SegmentGenerationUtils; import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator; import org.apache.pinot.controller.helix.core.minion.generator.TaskGeneratorUtils; @@ -48,6 +46,7 @@ import org.apache.pinot.core.common.MinionConstants.MergeTask; import org.apache.pinot.core.minion.PinotTaskConfig; import org.apache.pinot.plugin.minion.tasks.MergeTaskUtils; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; import org.apache.pinot.spi.annotations.minion.TaskGenerator; import org.apache.pinot.spi.config.table.ColumnPartitionConfig; import org.apache.pinot.spi.config.table.SegmentPartitionConfig; @@ -108,8 +107,6 @@ public class MergeRollupTaskGenerator extends BaseTaskGenerator { private static final int DEFAULT_MAX_NUM_RECORDS_PER_TASK = 50_000_000; private static final int DEFAULT_NUM_PARALLEL_BUCKETS = 1; private static final String REFRESH = "REFRESH"; - private static final BatchConfigProperties.SegmentPushType DEFAULT_SEGMENT_PUSH_TYPE = - BatchConfigProperties.SegmentPushType.TAR; // This is the metric that keeps track of the task delay in the number of time buckets. For example, if we see this // number to be 7 and merge task is configured with "bucketTimePeriod = 1d", this means that we have 7 days of @@ -546,7 +543,8 @@ private List createPinotTaskConfigs(List sel List pinotTaskConfigs = new ArrayList<>(); for (int i = 0; i < segmentNamesList.size(); i++) { String downloadURL = StringUtils.join(downloadURLsList.get(i), MinionConstants.URL_SEPARATOR); - Map configs = getPushTaskConfig(taskConfigs, downloadURL); + Map configs = MinionTaskUtils.getPushTaskConfig(offlineTableName, taskConfigs, + _clusterInfoAccessor); configs.put(MinionConstants.TABLE_NAME_KEY, offlineTableName); configs.put(MinionConstants.SEGMENT_NAME_KEY, StringUtils.join(segmentNamesList.get(i), MinionConstants.SEGMENT_NAME_SEPARATOR)); @@ -578,40 +576,6 @@ private List createPinotTaskConfigs(List sel return pinotTaskConfigs; } - private Map getPushTaskConfig(Map batchConfigMap, String downloadUrls) { - try { - String[] downloadURLList = downloadUrls.split(MinionConstants.URL_SEPARATOR); - if (downloadURLList.length > 0) { - String downloadUrl = downloadURLList[0]; - URI downloadURI = URI.create(downloadUrl); - URI outputDirURI = null; - if (!downloadURI.getScheme().contentEquals("http")) { - String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); - outputDirURI = URI.create(outputDir); - } - String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); - - Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); - if (outputDirURI != null) { - URI outputSegmentDirURI = SegmentGenerationUtils.getRelativeOutputPath( - outputDirURI, downloadURI, outputDirURI); - singleFileGenerationTaskConfig.put( - BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputSegmentDirURI.toString()); - } - if ((outputDirURI == null) || (pushMode == null)) { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); - } else { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); - } - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl()); - return singleFileGenerationTaskConfig; - } - return batchConfigMap; - } catch (Exception e) { - return batchConfigMap; - } - } - private long getMergeRollupTaskDelayInNumTimeBuckets(long watermarkMs, long maxEndTimeMsOfCurrentLevel, long bufferTimeMs, long bucketTimeMs) { if (watermarkMs == -1 || maxEndTimeMsOfCurrentLevel == Long.MIN_VALUE) { diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java index 470856c995c7..0f8a72c24fbb 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java @@ -19,7 +19,6 @@ package org.apache.pinot.plugin.minion.tasks.realtimetoofflinesegments; import com.google.common.base.Preconditions; -import java.net.URI; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -31,7 +30,6 @@ import org.apache.helix.zookeeper.datamodel.ZNRecord; import org.apache.pinot.common.metadata.segment.SegmentZKMetadata; import org.apache.pinot.common.minion.RealtimeToOfflineSegmentsTaskMetadata; -import org.apache.pinot.common.segment.generation.SegmentGenerationUtils; import org.apache.pinot.common.utils.LLCSegmentName; import org.apache.pinot.controller.helix.core.minion.generator.BaseTaskGenerator; import org.apache.pinot.controller.helix.core.minion.generator.PinotTaskGenerator; @@ -39,6 +37,7 @@ import org.apache.pinot.core.common.MinionConstants; import org.apache.pinot.core.common.MinionConstants.RealtimeToOfflineSegmentsTask; import org.apache.pinot.core.minion.PinotTaskConfig; +import org.apache.pinot.plugin.minion.tasks.MinionTaskUtils; import org.apache.pinot.spi.annotations.minion.TaskGenerator; import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableTaskConfig; @@ -209,8 +208,8 @@ public List generateTasks(List tableConfigs) { continue; } - Map configs = getPushTaskConfig(realtimeTableName, taskConfigs, - StringUtils.join(downloadURLs, MinionConstants.URL_SEPARATOR)); + Map configs = MinionTaskUtils.getPushTaskConfig(realtimeTableName, taskConfigs, + _clusterInfoAccessor); configs.put(MinionConstants.TABLE_NAME_KEY, realtimeTableName); configs.put(MinionConstants.SEGMENT_NAME_KEY, StringUtils.join(segmentNames, ",")); configs.put(MinionConstants.DOWNLOAD_URL_KEY, StringUtils.join(downloadURLs, MinionConstants.URL_SEPARATOR)); @@ -327,37 +326,4 @@ private long getWatermarkMs(String realtimeTableName, List co } return realtimeToOfflineSegmentsTaskMetadata.getWatermarkMs(); } - - private Map getPushTaskConfig(String tableName, - Map batchConfigMap, String downloadUrls) { - try { - String[] downloadURLList = downloadUrls.split(MinionConstants.URL_SEPARATOR); - if (downloadURLList.length > 0) { - String downloadUrl = downloadURLList[0]; - URI downloadURI = URI.create(downloadUrl); - URI outputDirURI = null; - if (!downloadURI.getScheme().contentEquals("http")) { - String outputDir = downloadUrl.substring(0, downloadUrl.lastIndexOf("/")); - outputDirURI = SegmentGenerationUtils.getDirectoryURI(outputDir); - } else { - outputDirURI = SegmentGenerationUtils.getDirectoryURI(_clusterInfoAccessor.getDataDir() + "/" + tableName); - } - String pushMode = IngestionConfigUtils.getPushMode(batchConfigMap); - - Map singleFileGenerationTaskConfig = new HashMap<>(batchConfigMap); - singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputDirURI.toString()); - if (pushMode == null) { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); - } else { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); - } - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, _clusterInfoAccessor.getVipUrl()); - return singleFileGenerationTaskConfig; - } - return batchConfigMap; - } catch (Exception e) { - LOGGER.warn("Error occurred while generating push task config", e); - return batchConfigMap; - } - } } From 1ffb275ff57eec68e5c357144ca892e9652df231 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Wed, 30 Nov 2022 20:09:27 +0530 Subject: [PATCH 13/16] Remove redundant code for URI push --- .../tasks/BaseMultipleSegmentsConversionExecutor.java | 11 ----------- .../RealtimeToOfflineSegmentsTaskGenerator.java | 3 --- 2 files changed, 14 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 0ae2f194fbca..fc4f180bcb51 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -327,17 +327,6 @@ private void pushSegment(String tableName, Map taskConfigs, URI throw new RuntimeException(e); } break; - case URI: - try { - List segmentUris = new ArrayList<>(); - URI updatedURI = SegmentPushUtils.generateSegmentTarURI(outputSegmentDirURI, outputSegmentTarURI, - pushJobSpec.getSegmentUriPrefix(), pushJobSpec.getSegmentUriSuffix()); - segmentUris.add(updatedURI.toString()); - SegmentPushUtils.sendSegmentUris(spec, segmentUris, headers, parameters); - } catch (RetriableOperationException | AttemptsExceededException e) { - throw new RuntimeException(e); - } - break; case METADATA: try { Map segmentUriToTarPathMap = diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java index 0f8a72c24fbb..ff65c704ba1a 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/realtimetoofflinesegments/RealtimeToOfflineSegmentsTaskGenerator.java @@ -42,7 +42,6 @@ import org.apache.pinot.spi.config.table.TableConfig; import org.apache.pinot.spi.config.table.TableTaskConfig; import org.apache.pinot.spi.config.table.TableType; -import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.stream.StreamConfig; import org.apache.pinot.spi.utils.CommonConstants.Segment; import org.apache.pinot.spi.utils.IngestionConfigUtils; @@ -84,8 +83,6 @@ public class RealtimeToOfflineSegmentsTaskGenerator extends BaseTaskGenerator { private static final String DEFAULT_BUCKET_PERIOD = "1d"; private static final String DEFAULT_BUFFER_PERIOD = "2d"; - private static final BatchConfigProperties.SegmentPushType DEFAULT_SEGMENT_PUSH_TYPE = - BatchConfigProperties.SegmentPushType.TAR; @Override public String getTaskType() { From 72efe9e68f5d26737e897b84ec0bcdd3dcd318ab Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Fri, 2 Dec 2022 19:17:38 +0530 Subject: [PATCH 14/16] Enforce outputDir in metadata push --- ...rgeRollupMinionClusterIntegrationTest.java | 3 +- ...aseMultipleSegmentsConversionExecutor.java | 48 +++++++++---------- .../plugin/minion/tasks/MinionTaskUtils.java | 34 ++++++++----- 3 files changed, 47 insertions(+), 38 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java index 3c27188b7fa6..96022750688f 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/MergeRollupMinionClusterIntegrationTest.java @@ -90,8 +90,7 @@ public class MergeRollupMinionClusterIntegrationTest extends BaseClusterIntegrat @BeforeClass public void setUp() throws Exception { - TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, - _segmentDir1, _segmentDir2, _segmentDir3, _segmentDir4, + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir1, _segmentDir2, _segmentDir3, _segmentDir4, _tarDir1, _tarDir2, _tarDir3, _tarDir4); // Start the Pinot cluster diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index fc4f180bcb51..8279a077778a 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -24,6 +24,7 @@ import java.net.URI; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -313,22 +314,19 @@ private void pushSegment(String tableName, Map taskConfigs, URI SegmentGenerationJobSpec spec = generatePushJobSpec(tableName, taskConfigs, pushJobSpec); - URI outputSegmentDirURI = null; - if (taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { - outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); - } - try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { - switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { - case TAR: - try (PinotFS pinotFS = MinionTaskUtils.getLocalPinotFs()) { - SegmentPushUtils.pushSegments( - spec, pinotFS, Arrays.asList(outputSegmentTarURI.toString()), headers, parameters); - } catch (RetriableOperationException | AttemptsExceededException e) { - throw new RuntimeException(e); - } - break; - case METADATA: - try { + switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { + case TAR: + try (PinotFS pinotFS = MinionTaskUtils.getLocalPinotFs()) { + SegmentPushUtils.pushSegments( + spec, pinotFS, Collections.singletonList(outputSegmentTarURI.toString()), headers, parameters); + } catch (RetriableOperationException | AttemptsExceededException e) { + throw new RuntimeException(e); + } + break; + case METADATA: + if (taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { + URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); + try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { Map segmentUriToTarPathMap = SegmentPushUtils.getSegmentUriToTarPathMap(outputSegmentDirURI, pushJobSpec, new String[]{outputSegmentTarURI.toString()}); @@ -336,10 +334,12 @@ private void pushSegment(String tableName, Map taskConfigs, URI } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); } - break; - default: - throw new UnsupportedOperationException("Unrecognized push mode - " + pushMode); - } + } else { + throw new RuntimeException("Output dir URI missing for metadata push"); + } + break; + default: + throw new UnsupportedOperationException("Unrecognized push mode - " + pushMode); } } @@ -364,15 +364,13 @@ private SegmentGenerationJobSpec generatePushJobSpec(String tableName, Map taskConfigs, File localSegmentTarFile) throws Exception { - if (!taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { - return localSegmentTarFile.toURI(); - } URI outputSegmentDirURI = URI.create(taskConfigs.get(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)); try (PinotFS outputFileFS = MinionTaskUtils.getOutputPinotFS(taskConfigs, outputSegmentDirURI)) { URI outputSegmentTarURI = URI.create(outputSegmentDirURI + localSegmentTarFile.getName()); if (!Boolean.parseBoolean(taskConfigs.get(BatchConfigProperties.OVERWRITE_OUTPUT)) && outputFileFS.exists( - outputSegmentDirURI)) { - LOGGER.warn("Not overwrite existing output segment tar file: {}", outputFileFS.exists(outputSegmentDirURI)); + outputSegmentTarURI)) { + throw new RuntimeException(String.format("Output file: %s already exists. " + + "Set 'overwriteOutput' to true to ignore this error", outputSegmentTarURI)); } else { outputFileFS.copyFromLocalFile(localSegmentTarFile, outputSegmentTarURI); } diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java index 2ed6e4b1d17c..0d8c1e375c54 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/MinionTaskUtils.java @@ -29,11 +29,12 @@ import org.apache.pinot.spi.ingestion.batch.BatchConfigProperties; import org.apache.pinot.spi.plugin.PluginManager; import org.apache.pinot.spi.utils.IngestionConfigUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class MinionTaskUtils { - private static final BatchConfigProperties.SegmentPushType DEFAULT_SEGMENT_PUSH_TYPE = - BatchConfigProperties.SegmentPushType.TAR; + private static final Logger LOGGER = LoggerFactory.getLogger(MinionTaskUtils.class); private MinionTaskUtils() { } @@ -75,19 +76,30 @@ public static PinotFS getOutputPinotFS(Map taskConfigs, URI file public static Map getPushTaskConfig(String tableName, Map taskConfigs, ClusterInfoAccessor clusterInfoAccessor) { try { - URI outputDirURI = URI.create(clusterInfoAccessor.getDataDir() + "/" + tableName); - String outputDirURIScheme = outputDirURI.getScheme(); String pushMode = IngestionConfigUtils.getPushMode(taskConfigs); Map singleFileGenerationTaskConfig = new HashMap<>(taskConfigs); - if (!isLocalOutputDir(outputDirURIScheme)) { - singleFileGenerationTaskConfig.put( - BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputDirURI.toString()); - } - if (isLocalOutputDir(outputDirURIScheme) || (pushMode == null)) { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, DEFAULT_SEGMENT_PUSH_TYPE.toString()); + if (pushMode == null + || pushMode.toUpperCase().contentEquals(BatchConfigProperties.SegmentPushType.TAR.toString())) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, + BatchConfigProperties.SegmentPushType.TAR.toString()); } else { - singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + URI outputDirURI = URI.create(clusterInfoAccessor.getDataDir() + "/" + tableName); + String outputDirURIScheme = outputDirURI.getScheme(); + + if (!isLocalOutputDir(outputDirURIScheme)) { + singleFileGenerationTaskConfig.put(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI, outputDirURI.toString()); + if (pushMode.toUpperCase().contentEquals(BatchConfigProperties.SegmentPushType.URI.toString())) { + LOGGER.warn("URI push type is not supported in this task. Switching to METADATA push"); + pushMode = BatchConfigProperties.SegmentPushType.METADATA.toString(); + } + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, pushMode); + } else { + LOGGER.warn("segment upload with METADATA push is not supported with local output dir: {}." + + " Switching to TAR push.", outputDirURI); + singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_MODE, + BatchConfigProperties.SegmentPushType.TAR.toString()); + } } singleFileGenerationTaskConfig.put(BatchConfigProperties.PUSH_CONTROLLER_URI, clusterInfoAccessor.getVipUrl()); return singleFileGenerationTaskConfig; From 8b4dd1a699581c8b44142818ff2977f59dc4f336 Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Tue, 6 Dec 2022 00:43:52 +0530 Subject: [PATCH 15/16] Use SegmentConversionUtils method to push TAR files for backward compatibility --- .../BaseMultipleSegmentsConversionExecutor.java | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 8279a077778a..00f250d637f8 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -24,7 +24,6 @@ import java.net.URI; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -279,7 +278,8 @@ public List executeTask(PinotTaskConfig pinotTaskConfig TableNameBuilder.extractRawTableName(tableNameWithType)); List parameters = Arrays.asList(enableParallelPushProtectionParameter, tableNameParameter); - pushSegment(tableNameParameter.getValue(), configs, outputSegmentTarURI, httpHeaders, parameters); + pushSegment(tableNameParameter.getValue(), configs, outputSegmentTarURI, httpHeaders, parameters, + segmentConversionResult); if (!FileUtils.deleteQuietly(convertedTarredSegmentFile)) { LOGGER.warn("Failed to delete tarred converted segment: {}", convertedTarredSegmentFile.getAbsolutePath()); } @@ -301,7 +301,8 @@ public List executeTask(PinotTaskConfig pinotTaskConfig } private void pushSegment(String tableName, Map taskConfigs, URI outputSegmentTarURI, - List
headers, List parameters) throws Exception { + List
headers, List parameters, SegmentConversionResult segmentConversionResult) + throws Exception { String pushMode = taskConfigs.get(BatchConfigProperties.PUSH_MODE); LOGGER.info("Trying to push Pinot segment with push mode {} from {}", pushMode, outputSegmentTarURI); @@ -316,9 +317,13 @@ private void pushSegment(String tableName, Map taskConfigs, URI switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { case TAR: - try (PinotFS pinotFS = MinionTaskUtils.getLocalPinotFs()) { - SegmentPushUtils.pushSegments( - spec, pinotFS, Collections.singletonList(outputSegmentTarURI.toString()), headers, parameters); + try { + File tarFile = new File(outputSegmentTarURI); + String segmentName = segmentConversionResult.getSegmentName(); + String tableNameWithType = segmentConversionResult.getTableNameWithType(); + String uploadURL = taskConfigs.get(MinionConstants.UPLOAD_URL_KEY); + SegmentConversionUtils.uploadSegment(taskConfigs, headers, parameters, tableNameWithType, segmentName, + uploadURL, tarFile); } catch (RetriableOperationException | AttemptsExceededException e) { throw new RuntimeException(e); } From ee76795c2ebfd951ad78a416016e801ef3fd432f Mon Sep 17 00:00:00 2001 From: Kartik Khare Date: Tue, 6 Dec 2022 15:30:29 +0530 Subject: [PATCH 16/16] Do not catch exception since it is already handled --- .../tasks/BaseMultipleSegmentsConversionExecutor.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java index 00f250d637f8..810f07b6d885 100644 --- a/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java +++ b/pinot-plugins/pinot-minion-tasks/pinot-minion-builtin-tasks/src/main/java/org/apache/pinot/plugin/minion/tasks/BaseMultipleSegmentsConversionExecutor.java @@ -56,8 +56,6 @@ import org.apache.pinot.spi.ingestion.batch.spec.SegmentGenerationJobSpec; import org.apache.pinot.spi.ingestion.batch.spec.TableSpec; import org.apache.pinot.spi.utils.builder.TableNameBuilder; -import org.apache.pinot.spi.utils.retry.AttemptsExceededException; -import org.apache.pinot.spi.utils.retry.RetriableOperationException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -317,16 +315,12 @@ private void pushSegment(String tableName, Map taskConfigs, URI switch (BatchConfigProperties.SegmentPushType.valueOf(pushMode.toUpperCase())) { case TAR: - try { File tarFile = new File(outputSegmentTarURI); String segmentName = segmentConversionResult.getSegmentName(); String tableNameWithType = segmentConversionResult.getTableNameWithType(); String uploadURL = taskConfigs.get(MinionConstants.UPLOAD_URL_KEY); SegmentConversionUtils.uploadSegment(taskConfigs, headers, parameters, tableNameWithType, segmentName, uploadURL, tarFile); - } catch (RetriableOperationException | AttemptsExceededException e) { - throw new RuntimeException(e); - } break; case METADATA: if (taskConfigs.containsKey(BatchConfigProperties.OUTPUT_SEGMENT_DIR_URI)) { @@ -336,8 +330,6 @@ private void pushSegment(String tableName, Map taskConfigs, URI SegmentPushUtils.getSegmentUriToTarPathMap(outputSegmentDirURI, pushJobSpec, new String[]{outputSegmentTarURI.toString()}); SegmentPushUtils.sendSegmentUriAndMetadata(spec, outputFileFS, segmentUriToTarPathMap, headers, parameters); - } catch (RetriableOperationException | AttemptsExceededException e) { - throw new RuntimeException(e); } } else { throw new RuntimeException("Output dir URI missing for metadata push");