From a7a6fe065d46ce1166b2fd51a0c6268cafc7285f Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Tue, 6 Apr 2021 20:12:52 -0400 Subject: [PATCH 01/20] Do stuff --- .../druid/indexer/DataSegmentAndTmpPath.java | 57 +++++++ .../indexer/HadoopDruidIndexerConfig.java | 2 +- .../druid/indexer/HadoopDruidIndexerJob.java | 11 +- .../druid/indexer/IndexGeneratorJob.java | 20 +-- .../org/apache/druid/indexer/JobHelper.java | 87 +--------- .../indexer/MetadataStorageUpdaterJob.java | 4 +- .../druid/indexer/IndexGeneratorJobTest.java | 6 +- .../indexing/common/task/HadoopIndexTask.java | 159 ++++++++++++++++-- 8 files changed, 235 insertions(+), 111 deletions(-) create mode 100644 indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java new file mode 100644 index 000000000000..00a23e76b337 --- /dev/null +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java @@ -0,0 +1,57 @@ +package org.apache.druid.indexer; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.DataSegment; + +import java.util.Objects; + +public class DataSegmentAndTmpPath +{ + private final DataSegment segment; + private final String indexZipFilePath; + + public DataSegmentAndTmpPath( + DataSegment segment, + String indexZipFilePath + ) + { + this.segment = segment; + this.indexZipFilePath = indexZipFilePath; + } + + @JsonProperty + public DataSegment getSegment() + { + return segment; + } + + @JsonProperty + public String getIndexZipFilePath() + { + return indexZipFilePath; + } + + @Override + public boolean equals(Object o) + { + if (o instanceof DataSegmentAndTmpPath) { + return segment.getId().equals(((DataSegmentAndTmpPath) o).getSegment().getId()); + } + return false; + } + + @Override + public int hashCode() + { + return Objects.hash(segment.getId(), indexZipFilePath); + } + + @Override + public String toString() + { + return "DataSegmentAndTmpPath{" + + "segment=" + segment + + ", indexZipFilePath=" + indexZipFilePath + + '}'; + } +} diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java index 54c8f073a9b9..cf6f34220c6a 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java @@ -578,7 +578,7 @@ void addJobProperties(Job job) addJobProperties(job.getConfiguration()); } - void addJobProperties(Configuration conf) + public void addJobProperties(Configuration conf) { for (final Map.Entry entry : schema.getTuningConfig().getJobProperties().entrySet()) { conf.set(entry.getKey(), entry.getValue()); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java index 25683f32bd8f..c5b5684783a8 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java @@ -22,7 +22,6 @@ import com.google.common.base.Preconditions; import com.google.inject.Inject; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.timeline.DataSegment; import javax.annotation.Nullable; import java.util.ArrayList; @@ -40,7 +39,7 @@ public class HadoopDruidIndexerJob implements Jobby @Nullable private IndexGeneratorJob indexJob; @Nullable - private volatile List publishedSegments = null; + private volatile List publishedSegmentAndTmpPaths = null; @Nullable private String hadoopJobIdFile; @@ -91,7 +90,7 @@ public boolean run() @Override public boolean run() { - publishedSegments = IndexGeneratorJob.getPublishedSegments(config); + publishedSegmentAndTmpPaths = IndexGeneratorJob.getPublishedSegmentAndTmpPaths(config); return true; } } @@ -122,12 +121,12 @@ public String getErrorMessage() return indexJob.getErrorMessage(); } - public List getPublishedSegments() + public List getPublishedSegmentAndTmpPaths() { - if (publishedSegments == null) { + if (publishedSegmentAndTmpPaths == null) { throw new IllegalStateException("Job hasn't run yet. No segments have been published yet."); } - return publishedSegments; + return publishedSegmentAndTmpPaths; } public void setHadoopJobIdFile(String hadoopJobIdFile) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index a12e76571dba..212e971e2adc 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -102,14 +102,14 @@ public class IndexGeneratorJob implements Jobby { private static final Logger log = new Logger(IndexGeneratorJob.class); - public static List getPublishedSegments(HadoopDruidIndexerConfig config) + public static List getPublishedSegmentAndTmpPaths(HadoopDruidIndexerConfig config) { final Configuration conf = JobHelper.injectSystemProperties(new Configuration(), config); config.addJobProperties(conf); final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.JSON_MAPPER; - ImmutableList.Builder publishedSegmentsBuilder = ImmutableList.builder(); + ImmutableList.Builder publishedSegmentAndTmpPathsBuilder = ImmutableList.builder(); final Path descriptorInfoDir = config.makeDescriptorInfoDir(); @@ -117,9 +117,9 @@ public static List getPublishedSegments(HadoopDruidIndexerConfig co FileSystem fs = descriptorInfoDir.getFileSystem(conf); for (FileStatus status : fs.listStatus(descriptorInfoDir)) { - final DataSegment segment = jsonMapper.readValue((InputStream) fs.open(status.getPath()), DataSegment.class); - publishedSegmentsBuilder.add(segment); - log.info("Adding segment %s to the list of published segments", segment.getId()); + final DataSegmentAndTmpPath segmentAndTmpPath = jsonMapper.readValue((InputStream) fs.open(status.getPath()), DataSegmentAndTmpPath.class); + publishedSegmentAndTmpPathsBuilder.add(segmentAndTmpPath); + log.info("Adding segment %s to the list of published segments", segmentAndTmpPath.getSegment().getId()); } } catch (FileNotFoundException e) { @@ -133,9 +133,9 @@ public static List getPublishedSegments(HadoopDruidIndexerConfig co catch (IOException e) { throw new RuntimeException(e); } - List publishedSegments = publishedSegmentsBuilder.build(); + List publishedSegmentAndTmpPaths = publishedSegmentAndTmpPathsBuilder.build(); - return publishedSegments; + return publishedSegmentAndTmpPaths; } private final HadoopDruidIndexerConfig config; @@ -809,7 +809,7 @@ public void doRun() 0 ); - final DataSegment segment = JobHelper.serializeOutIndex( + final DataSegmentAndTmpPath segmentAndTmpPath = JobHelper.serializeOutIndex( segmentTemplate, context.getConfiguration(), context, @@ -831,7 +831,7 @@ public void doRun() HadoopDruidIndexerConfig.DATA_SEGMENT_PUSHER ); - Path descriptorPath = config.makeDescriptorInfoPath(segment); + Path descriptorPath = config.makeDescriptorInfoPath(segmentAndTmpPath.getSegment()); descriptorPath = JobHelper.prependFSIfNullScheme( FileSystem.get( descriptorPath.toUri(), @@ -842,7 +842,7 @@ public void doRun() log.info("Writing descriptor to path[%s]", descriptorPath); JobHelper.writeSegmentDescriptor( config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration()), - segment, + segmentAndTmpPath, descriptorPath, context ); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index 7d99d03ad565..92ddf3d474da 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -436,7 +436,7 @@ public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) return succeeded; } - public static DataSegment serializeOutIndex( + public static DataSegmentAndTmpPath serializeOutIndex( final DataSegment segmentTemplate, final Configuration configuration, final Progressable progressable, @@ -447,6 +447,9 @@ public static DataSegment serializeOutIndex( ) throws IOException { + log.info("in Reducer: finalIndexZipFilePath: [%s], tmpPath: [%s]", + finalIndexZipFilePath.toUri().getPath(), + tmpPath.toUri().getPath()); final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); final AtomicLong size = new AtomicLong(0L); final DataPusher zipPusher = (DataPusher) RetryProxy.create( @@ -482,20 +485,12 @@ public long push() throws IOException .withSize(size.get()) .withBinaryVersion(SegmentUtils.getVersionFromDir(mergedBase)); - if (!renameIndexFiles(outputFS, tmpPath, finalIndexZipFilePath)) { - throw new IOE( - "Unable to rename [%s] to [%s]", - tmpPath.toUri().toString(), - finalIndexZipFilePath.toUri().toString() - ); - } - - return finalSegment; + return new DataSegmentAndTmpPath(finalSegment, tmpPath.toUri().getPath()); } public static void writeSegmentDescriptor( final FileSystem outputFS, - final DataSegment segment, + final DataSegmentAndTmpPath segmentAndPath, final Path descriptorPath, final Progressable progressable ) @@ -521,7 +516,7 @@ public long push() throws IOException DEFAULT_FS_BUFFER_SIZE, progressable )) { - HadoopDruidIndexerConfig.JSON_MAPPER.writeValue(descriptorOut, segment); + HadoopDruidIndexerConfig.JSON_MAPPER.writeValue(descriptorOut, segmentAndPath); } } catch (RuntimeException | IOException ex) { @@ -631,74 +626,6 @@ public static Path makeTmpPath( ); } - /** - * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename - * which will not overwrite - * - * @param outputFS The output fs - * @param indexZipFilePath The original file path - * @param finalIndexZipFilePath The to rename the original file to - * - * @return False if a rename failed, true otherwise (rename success or no rename needed) - */ - private static boolean renameIndexFiles( - final FileSystem outputFS, - final Path indexZipFilePath, - final Path finalIndexZipFilePath - ) - { - try { - return RetryUtils.retry( - () -> { - final boolean needRename; - - if (outputFS.exists(finalIndexZipFilePath)) { - // NativeS3FileSystem.rename won't overwrite, so we might need to delete the old index first - final FileStatus zipFile = outputFS.getFileStatus(indexZipFilePath); - final FileStatus finalIndexZipFile = outputFS.getFileStatus(finalIndexZipFilePath); - - if (zipFile.getModificationTime() >= finalIndexZipFile.getModificationTime() - || zipFile.getLen() != finalIndexZipFile.getLen()) { - log.info( - "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", - finalIndexZipFile.getPath(), - DateTimes.utc(finalIndexZipFile.getModificationTime()), - finalIndexZipFile.getLen(), - zipFile.getPath(), - DateTimes.utc(zipFile.getModificationTime()), - zipFile.getLen() - ); - outputFS.delete(finalIndexZipFilePath, false); - needRename = true; - } else { - log.info( - "File[%s / %s / %sB] existed and will be kept", - finalIndexZipFile.getPath(), - DateTimes.utc(finalIndexZipFile.getModificationTime()), - finalIndexZipFile.getLen() - ); - needRename = false; - } - } else { - needRename = true; - } - - if (needRename) { - log.info("Attempting rename from [%s] to [%s]", indexZipFilePath, finalIndexZipFilePath); - return outputFS.rename(indexZipFilePath, finalIndexZipFilePath); - } else { - return true; - } - }, - FileUtils.IS_EXCEPTION, - NUM_RETRIES - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - public static Path prependFSIfNullScheme(FileSystem fs, Path path) { diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java index b7eb60bf0931..a2781faa9282 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java @@ -22,6 +22,7 @@ import org.apache.druid.timeline.DataSegment; import java.util.List; +import java.util.stream.Collectors; /** */ @@ -42,7 +43,8 @@ public MetadataStorageUpdaterJob( @Override public boolean run() { - final List segments = IndexGeneratorJob.getPublishedSegments(config); + final List segmentAndTmpPaths = IndexGeneratorJob.getPublishedSegmentAndTmpPaths(config); + final List segments = segmentAndTmpPaths.stream().map(s -> s.getSegment()).collect(Collectors.toList()); final String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable(); handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.JSON_MAPPER); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index c0254c4fbe43..067b3077bc9e 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -624,9 +624,9 @@ private void verifyJob(IndexGeneratorJob job) throws IOException final Map> intervalToSegments = new HashMap<>(); IndexGeneratorJob - .getPublishedSegments(config) - .forEach(segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()) - .add(segment)); + .getPublishedSegmentAndTmpPaths(config) + .forEach(segmentAndTmpPath -> intervalToSegments.computeIfAbsent(segmentAndTmpPath.getSegment().getInterval(), k -> new ArrayList<>()) + .add(segmentAndTmpPath.getSegment())); final Map> intervalToIndexFiles = new HashMap<>(); int segmentNum = 0; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index c11ce31c59a9..98efcfd04e09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -30,6 +30,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import org.apache.druid.indexer.DataSegmentAndTmpPath; import org.apache.druid.indexer.HadoopDruidDetermineConfigurationJob; import org.apache.druid.indexer.HadoopDruidIndexerConfig; import org.apache.druid.indexer.HadoopDruidIndexerJob; @@ -51,7 +52,11 @@ import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.hadoop.OverlordActionBasedUsedSegmentsRetriever; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; @@ -63,6 +68,9 @@ import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.util.ToolRunner; import org.joda.time.Interval; @@ -77,17 +85,21 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import java.io.File; +import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class HadoopIndexTask extends HadoopTask implements ChatHandler { private static final Logger log = new Logger(HadoopIndexTask.class); private static final String HADOOP_JOB_ID_FILENAME = "mapReduceJobId.json"; private static final String TYPE = "index_hadoop"; + private static final int NUM_RETRIES = 8; + public static final String INDEX_ZIP = "index.zip"; private TaskConfig taskConfig = null; private static String getTheDataSource(HadoopIngestionSpec spec) @@ -337,6 +349,9 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception "runTask", determinePartitionsInput.getClass() ); + + HadoopDruidIndexerConfig config; + try { Thread.currentThread().setContextClassLoader(loader); @@ -353,6 +368,8 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); indexerSchema = determineConfigStatus.getSchema(); + String workingPath = determineConfigStatus.getWorkingPath(); + String segmentOutputPath = determineConfigStatus.getSegmentOutputPath(); if (indexerSchema == null) { errorMsg = determineConfigStatus.getErrorMsg(); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); @@ -361,6 +378,11 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception errorMsg ); } + config = HadoopDruidIndexerConfig.fromSpec( + indexerSchema + .withIOConfig(indexerSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) + .withTuningConfig(indexerSchema.getTuningConfig().withWorkingPath(workingPath)) + ); } catch (Exception e) { throw new RuntimeException(e); @@ -369,6 +391,9 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Thread.currentThread().setContextClassLoader(oldLoader); } + final Configuration conf = JobHelper.injectSystemProperties(new Configuration(), config); + config.addJobProperties(conf); + // We should have a lock from before we started running only if interval was specified String version; if (determineIntervals) { @@ -438,9 +463,11 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception HadoopIndexGeneratorInnerProcessingStatus.class ); - if (buildSegmentsStatus.getDataSegments() != null) { + List dataSegmentAndTmpPaths = buildSegmentsStatus.getDataSegmentsAndTmpPaths(); + if (dataSegmentAndTmpPaths != null) { + renameIndexFilesForSegments(conf, dataSegmentAndTmpPaths); ingestionState = IngestionState.COMPLETED; - toolbox.publishSegments(buildSegmentsStatus.getDataSegments()); + toolbox.publishSegments(dataSegmentAndTmpPaths.stream().map(DataSegmentAndTmpPath::getSegment).collect(Collectors.toList())); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } else { @@ -460,6 +487,100 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } } + /** + * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename + * which will not overwrite + */ + private static void renameIndexFilesForSegments( + Configuration configuration, + List segmentsAndTmpPaths + ) throws IOException + { + for (DataSegmentAndTmpPath segmentAndTmpPath : segmentsAndTmpPaths) { + org.apache.hadoop.fs.Path tmpPath = new org.apache.hadoop.fs.Path(segmentAndTmpPath.getIndexZipFilePath()); + org.apache.hadoop.fs.Path finalIndexZipFilePath = new org.apache.hadoop.fs.Path(tmpPath.getParent(), INDEX_ZIP); + final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); + if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { + throw new IOE( + "Unable to rename [%s] to [%s]", + tmpPath.toUri().toString(), + finalIndexZipFilePath.toUri().toString() + ); + } + } + } + + /** + * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename + * which will not overwrite + * + * @param outputFS The output fs + * @param indexZipFilePath The original file path + * @param finalIndexZipFilePath The to rename the original file to + * + * @return False if a rename failed, true otherwise (rename success or no rename needed) + */ + private static boolean renameIndexFile( + final FileSystem outputFS, + final org.apache.hadoop.fs.Path indexZipFilePath, + final org.apache.hadoop.fs.Path finalIndexZipFilePath + ) + { + log.info("renameIndexFile: finalIndexZipFilePath: [%s], tmpPath: [%s]", + finalIndexZipFilePath.toUri().getPath(), + indexZipFilePath.toUri().getPath()); + try { + return RetryUtils.retry( + () -> { + final boolean needRename; + + if (outputFS.exists(finalIndexZipFilePath)) { + // NativeS3FileSystem.rename won't overwrite, so we might need to delete the old index first + final FileStatus zipFile = outputFS.getFileStatus(indexZipFilePath); + final FileStatus finalIndexZipFile = outputFS.getFileStatus(finalIndexZipFilePath); + + if (zipFile.getModificationTime() >= finalIndexZipFile.getModificationTime() + || zipFile.getLen() != finalIndexZipFile.getLen()) { + log.info( + "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", + finalIndexZipFile.getPath(), + DateTimes.utc(finalIndexZipFile.getModificationTime()), + finalIndexZipFile.getLen(), + zipFile.getPath(), + DateTimes.utc(zipFile.getModificationTime()), + zipFile.getLen() + ); + outputFS.delete(finalIndexZipFilePath, false); + needRename = true; + } else { + log.info( + "File[%s / %s / %sB] existed and will be kept", + finalIndexZipFile.getPath(), + DateTimes.utc(finalIndexZipFile.getModificationTime()), + finalIndexZipFile.getLen() + ); + needRename = false; + } + } else { + needRename = true; + } + + if (needRename) { + log.info("Attempting rename from [%s] to [%s]", indexZipFilePath, finalIndexZipFilePath); + return outputFS.rename(indexZipFilePath, finalIndexZipFilePath); + } else { + return true; + } + }, + FileUtils.IS_EXCEPTION, + NUM_RETRIES + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + private void killHadoopJob() { // To avoid issue of kill command once the ingestion task is actually completed @@ -650,11 +771,11 @@ public String runTask(String[] args) throws Exception log.info("Starting a hadoop determine configuration job..."); if (job.run()) { return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( - new HadoopDetermineConfigInnerProcessingStatus(config.getSchema(), job.getStats(), null) + new HadoopDetermineConfigInnerProcessingStatus(config.getSchema(), workingPath, segmentOutputPath, job.getStats(), null) ); } else { return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( - new HadoopDetermineConfigInnerProcessingStatus(null, job.getStats(), job.getErrorMessage()) + new HadoopDetermineConfigInnerProcessingStatus(null, null, null, job.getStats(), job.getErrorMessage()) ); } } @@ -707,7 +828,7 @@ public String runTask(String[] args) throws Exception if (job.run()) { return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( new HadoopIndexGeneratorInnerProcessingStatus( - job.getPublishedSegments(), + job.getPublishedSegmentAndTmpPaths(), job.getStats(), null ) @@ -777,26 +898,26 @@ public String[] runTask(String[] args) throws Exception public static class HadoopIndexGeneratorInnerProcessingStatus { - private final List dataSegments; + private final List dataSegmentsAndTmpPaths; private final Map metrics; private final String errorMsg; @JsonCreator public HadoopIndexGeneratorInnerProcessingStatus( - @JsonProperty("dataSegments") List dataSegments, + @JsonProperty("dataSegments") List dataSegmentAndTmpPaths, @JsonProperty("metrics") Map metrics, @JsonProperty("errorMsg") String errorMsg ) { - this.dataSegments = dataSegments; + this.dataSegmentsAndTmpPaths = dataSegmentAndTmpPaths; this.metrics = metrics; this.errorMsg = errorMsg; } @JsonProperty - public List getDataSegments() + public List getDataSegmentsAndTmpPaths() { - return dataSegments; + return dataSegmentsAndTmpPaths; } @JsonProperty @@ -815,17 +936,23 @@ public String getErrorMsg() public static class HadoopDetermineConfigInnerProcessingStatus { private final HadoopIngestionSpec schema; + private final String workingPath; + private final String segmentOutputPath; private final Map metrics; private final String errorMsg; @JsonCreator public HadoopDetermineConfigInnerProcessingStatus( @JsonProperty("schema") HadoopIngestionSpec schema, + @JsonProperty("workingPath") String workingPath, + @JsonProperty("segmentOutputPath") String segmentOutputPath, @JsonProperty("metrics") Map metrics, @JsonProperty("errorMsg") String errorMsg ) { this.schema = schema; + this.workingPath = workingPath; + this.segmentOutputPath = segmentOutputPath; this.metrics = metrics; this.errorMsg = errorMsg; } @@ -836,6 +963,18 @@ public HadoopIngestionSpec getSchema() return schema; } + @JsonProperty + public String getWorkingPath() + { + return workingPath; + } + + @JsonProperty + public String getSegmentOutputPath() + { + return segmentOutputPath; + } + @JsonProperty public Map getMetrics() { From dcd966302ff8465994725269a0cd687384f36d35 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 16:05:01 -0400 Subject: [PATCH 02/20] Do more stuff --- .../druid/indexer/DataSegmentAndTmpPath.java | 6 +- .../org/apache/druid/indexer/JobHelper.java | 110 ++++++++++++++ .../indexing/common/task/HadoopIndexTask.java | 137 +++--------------- 3 files changed, 132 insertions(+), 121 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java index 00a23e76b337..fdc9d28fbed1 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java @@ -1,5 +1,6 @@ package org.apache.druid.indexer; +import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.timeline.DataSegment; @@ -10,9 +11,10 @@ public class DataSegmentAndTmpPath private final DataSegment segment; private final String indexZipFilePath; + @JsonCreator public DataSegmentAndTmpPath( - DataSegment segment, - String indexZipFilePath + @JsonProperty("segment") DataSegment segment, + @JsonProperty("indexZipFilePath") String indexZipFilePath ) { this.segment = segment; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index 92ddf3d474da..cad99d3528a1 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -762,4 +762,114 @@ public static String getJobTrackerAddress(Configuration config) } return jobTrackerAddress; } + + /** + * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename + * which will not overwrite + */ + public static void renameIndexFilesForSegments( + HadoopIngestionSpec indexerSchema, + String segmentOutputPath, + String workingPath, + List segmentsAndTmpPaths + ) throws IOException + { + log.info("Building HadoopDruidIndexerConfig"); + HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec( + indexerSchema + .withIOConfig(indexerSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) + .withTuningConfig(indexerSchema.getTuningConfig().withWorkingPath(workingPath)) + ); + log.info("Building Configuration"); + final Configuration configuration = JobHelper.injectSystemProperties(new Configuration(), config); + config.addJobProperties(configuration); + JobHelper.injectDruidProperties(configuration, config); + log.info("Built Configuration"); + for (DataSegmentAndTmpPath segmentAndTmpPath : segmentsAndTmpPaths) { + log.info("tmpPath: [%s]", segmentAndTmpPath.getIndexZipFilePath()); + log.info("segmentId: [%s]", segmentAndTmpPath.getSegment().getId()); + Path tmpPath = new Path(segmentAndTmpPath.getIndexZipFilePath()); + Path finalIndexZipFilePath = new Path(tmpPath.getParent(), INDEX_ZIP); + final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); + log.info("about to rename segment index file"); + if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { + throw new IOE( + "Unable to rename [%s] to [%s]", + tmpPath.toUri().toString(), + finalIndexZipFilePath.toUri().toString() + ); + } + } + } + + /** + * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename + * which will not overwrite + * + * @param outputFS The output fs + * @param indexZipFilePath The original file path + * @param finalIndexZipFilePath The to rename the original file to + * + * @return False if a rename failed, true otherwise (rename success or no rename needed) + */ + public static boolean renameIndexFile( + final FileSystem outputFS, + final Path indexZipFilePath, + final Path finalIndexZipFilePath + ) + { + log.info("renameIndexFile: finalIndexZipFilePath: [%s], tmpPath: [%s]", + finalIndexZipFilePath.toUri().getPath(), + indexZipFilePath.toUri().getPath()); + try { + return RetryUtils.retry( + () -> { + final boolean needRename; + + if (outputFS.exists(finalIndexZipFilePath)) { + // NativeS3FileSystem.rename won't overwrite, so we might need to delete the old index first + final FileStatus zipFile = outputFS.getFileStatus(indexZipFilePath); + final FileStatus finalIndexZipFile = outputFS.getFileStatus(finalIndexZipFilePath); + + if (zipFile.getModificationTime() >= finalIndexZipFile.getModificationTime() + || zipFile.getLen() != finalIndexZipFile.getLen()) { + log.info( + "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", + finalIndexZipFile.getPath(), + DateTimes.utc(finalIndexZipFile.getModificationTime()), + finalIndexZipFile.getLen(), + zipFile.getPath(), + DateTimes.utc(zipFile.getModificationTime()), + zipFile.getLen() + ); + outputFS.delete(finalIndexZipFilePath, false); + needRename = true; + } else { + log.info( + "File[%s / %s / %sB] existed and will be kept", + finalIndexZipFile.getPath(), + DateTimes.utc(finalIndexZipFile.getModificationTime()), + finalIndexZipFile.getLen() + ); + needRename = false; + } + } else { + needRename = true; + } + + if (needRename) { + log.info("Attempting rename from [%s] to [%s]", indexZipFilePath, finalIndexZipFilePath); + return outputFS.rename(indexZipFilePath, finalIndexZipFilePath); + } else { + return true; + } + }, + FileUtils.IS_EXCEPTION, + NUM_RETRIES + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 98efcfd04e09..d4c1350b07b5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -52,11 +52,7 @@ import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.hadoop.OverlordActionBasedUsedSegmentsRetriever; -import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.IOE; import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.RetryUtils; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.logger.Logger; @@ -68,9 +64,6 @@ import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.mapred.JobClient; import org.apache.hadoop.util.ToolRunner; import org.joda.time.Interval; @@ -85,7 +78,6 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import java.io.File; -import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.HashMap; @@ -343,6 +335,8 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception }; HadoopIngestionSpec indexerSchema; + String workingPath; + String segmentOutputPath; final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); Class determinePartitionsRunnerClass = determinePartitionsInnerProcessingRunner.getClass(); Method determinePartitionsInnerProcessingRunTask = determinePartitionsRunnerClass.getMethod( @@ -368,8 +362,8 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); indexerSchema = determineConfigStatus.getSchema(); - String workingPath = determineConfigStatus.getWorkingPath(); - String segmentOutputPath = determineConfigStatus.getSegmentOutputPath(); + workingPath = determineConfigStatus.getWorkingPath(); + segmentOutputPath = determineConfigStatus.getSegmentOutputPath(); if (indexerSchema == null) { errorMsg = determineConfigStatus.getErrorMsg(); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); @@ -378,11 +372,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception errorMsg ); } - config = HadoopDruidIndexerConfig.fromSpec( - indexerSchema - .withIOConfig(indexerSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) - .withTuningConfig(indexerSchema.getTuningConfig().withWorkingPath(workingPath)) - ); + log.info("workingPath: [%s], segmentOutputPath: [%s]", workingPath, segmentOutputPath); } catch (Exception e) { throw new RuntimeException(e); @@ -391,9 +381,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Thread.currentThread().setContextClassLoader(oldLoader); } - final Configuration conf = JobHelper.injectSystemProperties(new Configuration(), config); - config.addJobProperties(conf); - // We should have a lock from before we started running only if interval was specified String version; if (determineIntervals) { @@ -452,6 +439,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception try { Thread.currentThread().setContextClassLoader(loader); + log.info("about to invoke job"); ingestionState = IngestionState.BUILD_SEGMENTS; final String jobStatusString = (String) innerProcessingRunTask.invoke( innerProcessingRunner, @@ -463,14 +451,18 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception HadoopIndexGeneratorInnerProcessingStatus.class ); - List dataSegmentAndTmpPaths = buildSegmentsStatus.getDataSegmentsAndTmpPaths(); + log.info("about to get segment files"); + List dataSegmentAndTmpPaths = buildSegmentsStatus.getDataSegmentAndTmpPaths(); + log.info("about to rename segment files"); if (dataSegmentAndTmpPaths != null) { - renameIndexFilesForSegments(conf, dataSegmentAndTmpPaths); + log.info("found non-null segment files"); + JobHelper.renameIndexFilesForSegments(indexerSchema, segmentOutputPath, workingPath, dataSegmentAndTmpPaths); ingestionState = IngestionState.COMPLETED; toolbox.publishSegments(dataSegmentAndTmpPaths.stream().map(DataSegmentAndTmpPath::getSegment).collect(Collectors.toList())); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } else { + log.info("found null segment files :("); errorMsg = buildSegmentsStatus.getErrorMsg(); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( @@ -487,100 +479,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } } - /** - * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename - * which will not overwrite - */ - private static void renameIndexFilesForSegments( - Configuration configuration, - List segmentsAndTmpPaths - ) throws IOException - { - for (DataSegmentAndTmpPath segmentAndTmpPath : segmentsAndTmpPaths) { - org.apache.hadoop.fs.Path tmpPath = new org.apache.hadoop.fs.Path(segmentAndTmpPath.getIndexZipFilePath()); - org.apache.hadoop.fs.Path finalIndexZipFilePath = new org.apache.hadoop.fs.Path(tmpPath.getParent(), INDEX_ZIP); - final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); - if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { - throw new IOE( - "Unable to rename [%s] to [%s]", - tmpPath.toUri().toString(), - finalIndexZipFilePath.toUri().toString() - ); - } - } - } - - /** - * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename - * which will not overwrite - * - * @param outputFS The output fs - * @param indexZipFilePath The original file path - * @param finalIndexZipFilePath The to rename the original file to - * - * @return False if a rename failed, true otherwise (rename success or no rename needed) - */ - private static boolean renameIndexFile( - final FileSystem outputFS, - final org.apache.hadoop.fs.Path indexZipFilePath, - final org.apache.hadoop.fs.Path finalIndexZipFilePath - ) - { - log.info("renameIndexFile: finalIndexZipFilePath: [%s], tmpPath: [%s]", - finalIndexZipFilePath.toUri().getPath(), - indexZipFilePath.toUri().getPath()); - try { - return RetryUtils.retry( - () -> { - final boolean needRename; - - if (outputFS.exists(finalIndexZipFilePath)) { - // NativeS3FileSystem.rename won't overwrite, so we might need to delete the old index first - final FileStatus zipFile = outputFS.getFileStatus(indexZipFilePath); - final FileStatus finalIndexZipFile = outputFS.getFileStatus(finalIndexZipFilePath); - - if (zipFile.getModificationTime() >= finalIndexZipFile.getModificationTime() - || zipFile.getLen() != finalIndexZipFile.getLen()) { - log.info( - "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", - finalIndexZipFile.getPath(), - DateTimes.utc(finalIndexZipFile.getModificationTime()), - finalIndexZipFile.getLen(), - zipFile.getPath(), - DateTimes.utc(zipFile.getModificationTime()), - zipFile.getLen() - ); - outputFS.delete(finalIndexZipFilePath, false); - needRename = true; - } else { - log.info( - "File[%s / %s / %sB] existed and will be kept", - finalIndexZipFile.getPath(), - DateTimes.utc(finalIndexZipFile.getModificationTime()), - finalIndexZipFile.getLen() - ); - needRename = false; - } - } else { - needRename = true; - } - - if (needRename) { - log.info("Attempting rename from [%s] to [%s]", indexZipFilePath, finalIndexZipFilePath); - return outputFS.rename(indexZipFilePath, finalIndexZipFilePath); - } else { - return true; - } - }, - FileUtils.IS_EXCEPTION, - NUM_RETRIES - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - private void killHadoopJob() { // To avoid issue of kill command once the ingestion task is actually completed @@ -826,6 +724,7 @@ public String runTask(String[] args) throws Exception log.info("Starting a hadoop index generator job..."); try { if (job.run()) { + log.info("Constructing HadoopIndexGeneratorInnerProcessingStatus with segmentsAndPaths..."); return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( new HadoopIndexGeneratorInnerProcessingStatus( job.getPublishedSegmentAndTmpPaths(), @@ -898,26 +797,26 @@ public String[] runTask(String[] args) throws Exception public static class HadoopIndexGeneratorInnerProcessingStatus { - private final List dataSegmentsAndTmpPaths; + private final List dataSegmentAndTmpPaths; private final Map metrics; private final String errorMsg; @JsonCreator public HadoopIndexGeneratorInnerProcessingStatus( - @JsonProperty("dataSegments") List dataSegmentAndTmpPaths, + @JsonProperty("dataSegmentAndTmpPaths") List dataSegmentAndTmpPaths, @JsonProperty("metrics") Map metrics, @JsonProperty("errorMsg") String errorMsg ) { - this.dataSegmentsAndTmpPaths = dataSegmentAndTmpPaths; + this.dataSegmentAndTmpPaths = dataSegmentAndTmpPaths; this.metrics = metrics; this.errorMsg = errorMsg; } @JsonProperty - public List getDataSegmentsAndTmpPaths() + public List getDataSegmentAndTmpPaths() { - return dataSegmentsAndTmpPaths; + return dataSegmentAndTmpPaths; } @JsonProperty From fd26c73ee4f73a4c995d76550b479f56bdcf3405 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 18:04:26 -0400 Subject: [PATCH 03/20] * Do more stuff --- .../indexing/common/task/HadoopIndexTask.java | 121 +++++++++++++++++- 1 file changed, 119 insertions(+), 2 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index d4c1350b07b5..c52eb5cbf765 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; import com.google.common.base.Preconditions; @@ -456,9 +457,19 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception log.info("about to rename segment files"); if (dataSegmentAndTmpPaths != null) { log.info("found non-null segment files"); - JobHelper.renameIndexFilesForSegments(indexerSchema, segmentOutputPath, workingPath, dataSegmentAndTmpPaths); + + + renameSegmentIndexFilesJob( + indexerSchema, + segmentOutputPath, + workingPath, + dataSegmentAndTmpPaths + ); + ingestionState = IngestionState.COMPLETED; - toolbox.publishSegments(dataSegmentAndTmpPaths.stream().map(DataSegmentAndTmpPath::getSegment).collect(Collectors.toList())); + toolbox.publishSegments(dataSegmentAndTmpPaths.stream() + .map(DataSegmentAndTmpPath::getSegment) + .collect(Collectors.toList())); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } else { @@ -519,6 +530,64 @@ private void killHadoopJob() } } + private void renameSegmentIndexFilesJob( + HadoopIngestionSpec indexerSchema, + String segmentOutputPath, + String workingPath, + List dataSegmentAndTmpPaths + ) + { + log.info("In renameSegmentIndexFilesJob"); + final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); + try { + ClassLoader loader = HadoopTask.buildClassLoader( + getHadoopDependencyCoordinates(), + taskConfig.getDefaultHadoopCoordinates() + ); + + Object renameSegmentIndexFilesRunner = getForeignClassloaderObject( + "org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopRenameSegmentIndexFilesRunner", + loader + ); + + String hadoopIngestionSpecStr = + HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(indexerSchema); + String dataSegmentAndTmpPathListStr = + HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(dataSegmentAndTmpPaths); + String[] renameSegmentIndexFilesJobInput = new String[]{ + hadoopIngestionSpecStr, + workingPath, + segmentOutputPath, + dataSegmentAndTmpPathListStr + }; + + log.info( + "hadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndTmpPathListStr: [%s]", + hadoopIngestionSpecStr, + workingPath, + segmentOutputPath, + dataSegmentAndTmpPathListStr + ); + Class buildKillJobRunnerClass = renameSegmentIndexFilesRunner.getClass(); + Method renameSegmentIndexFiles = buildKillJobRunnerClass.getMethod( + "runTask", + renameSegmentIndexFilesJobInput.getClass() + ); + + Thread.currentThread().setContextClassLoader(loader); + renameSegmentIndexFiles.invoke( + renameSegmentIndexFilesRunner, + new Object[]{renameSegmentIndexFilesJobInput} + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } + } + @GET @Path("/rowStats") @Produces(MediaType.APPLICATION_JSON) @@ -795,6 +864,54 @@ public String[] runTask(String[] args) throws Exception } } + @SuppressWarnings("unused") + public static class HadoopRenameSegmentIndexFilesRunner + { + TypeReference> LIST_DATA_SEGMENT_AND_TMP_PATH = + new TypeReference>() + { + }; + + public void runTask(String[] args) throws Exception + { + if (args.length != 4) { + log.warn("HadoopRenameSegmentIndexFilesRunner called with improper number of arguments"); + } + String hadoopIngestionSpecStr = args[0]; + String workingPath = args[1]; + String segmentOutputPath = args[2]; + String dataSegmentAndTmpPathListStr = args[3]; + log.info( + "HadoopRenameSegmentIndexFilesRunner: HadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndTmpPathListStr: [%s]", + hadoopIngestionSpecStr, + workingPath, + segmentOutputPath, + dataSegmentAndTmpPathListStr + ); + + HadoopIngestionSpec indexerSchema; + List dataSegmentAndTmpPaths; + try { + indexerSchema = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( + hadoopIngestionSpecStr, + HadoopIngestionSpec.class + ); + dataSegmentAndTmpPaths = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( + dataSegmentAndTmpPathListStr, + LIST_DATA_SEGMENT_AND_TMP_PATH + ); + } + catch (Exception e) { + log.warn( + e, + "HadoopRenameSegmentIndexFilesRunner: Error occurred while trying to read input parameters into data objects" + ); + throw e; + } + JobHelper.renameIndexFilesForSegments(indexerSchema, segmentOutputPath, workingPath, dataSegmentAndTmpPaths); + } + } + public static class HadoopIndexGeneratorInnerProcessingStatus { private final List dataSegmentAndTmpPaths; From ab45103348f33b6be471ddbc1fc0bdf7c2a21f85 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 18:39:12 -0400 Subject: [PATCH 04/20] * Do more stuff --- .../indexing/common/task/HadoopIndexTask.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index c52eb5cbf765..69f0e0604e71 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -458,13 +458,18 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception if (dataSegmentAndTmpPaths != null) { log.info("found non-null segment files"); - - renameSegmentIndexFilesJob( - indexerSchema, - segmentOutputPath, - workingPath, - dataSegmentAndTmpPaths - ); + try { + Thread.currentThread().setContextClassLoader(oldLoader); + renameSegmentIndexFilesJob( + toolbox.getJsonMapper().writeValueAsString(indexerSchema), + segmentOutputPath, + workingPath, + toolbox.getJsonMapper().writeValueAsString(dataSegmentAndTmpPaths) + ); + } + finally { + Thread.currentThread().setContextClassLoader(loader); + } ingestionState = IngestionState.COMPLETED; toolbox.publishSegments(dataSegmentAndTmpPaths.stream() @@ -531,10 +536,10 @@ private void killHadoopJob() } private void renameSegmentIndexFilesJob( - HadoopIngestionSpec indexerSchema, + String hadoopIngestionSpecStr, String segmentOutputPath, String workingPath, - List dataSegmentAndTmpPaths + String dataSegmentAndTmpPathListStr ) { log.info("In renameSegmentIndexFilesJob"); @@ -550,10 +555,6 @@ private void renameSegmentIndexFilesJob( loader ); - String hadoopIngestionSpecStr = - HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(indexerSchema); - String dataSegmentAndTmpPathListStr = - HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(dataSegmentAndTmpPaths); String[] renameSegmentIndexFilesJobInput = new String[]{ hadoopIngestionSpecStr, workingPath, From 78f29586f61978c8844be6457960d412dea88e19 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 20:15:23 -0400 Subject: [PATCH 05/20] * working --- .../DataSegmentAndIndexZipFilePath.java | 69 +++++++++++++++++++ .../druid/indexer/DataSegmentAndTmpPath.java | 59 ---------------- .../druid/indexer/HadoopDruidIndexerJob.java | 4 +- .../druid/indexer/IndexGeneratorJob.java | 10 +-- .../org/apache/druid/indexer/JobHelper.java | 20 +++--- .../indexer/MetadataStorageUpdaterJob.java | 2 +- .../indexing/common/task/HadoopIndexTask.java | 54 +++++++-------- 7 files changed, 116 insertions(+), 102 deletions(-) create mode 100644 indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java delete mode 100644 indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java new file mode 100644 index 000000000000..f8b764863ea2 --- /dev/null +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java @@ -0,0 +1,69 @@ +package org.apache.druid.indexer; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.DataSegment; + +import java.util.Objects; + +public class DataSegmentAndIndexZipFilePath +{ + private final DataSegment segment; + private final String tmpIndexZipFilePath; + private final String finalIndexZipFilePath; + + @JsonCreator + public DataSegmentAndIndexZipFilePath( + @JsonProperty("segment") DataSegment segment, + @JsonProperty("tmpIndexZipFilePath") String tmpIndexZipFilePath, + @JsonProperty("finalIndexZipFilePath") String finalIndexZipFilePath + ) + { + this.segment = segment; + this.tmpIndexZipFilePath = tmpIndexZipFilePath; + this.finalIndexZipFilePath = finalIndexZipFilePath; + } + + @JsonProperty + public DataSegment getSegment() + { + return segment; + } + + @JsonProperty + public String getTmpIndexZipFilePath() + { + return tmpIndexZipFilePath; + } + + @JsonProperty + public String getFinalIndexZipFilePath() + { + return finalIndexZipFilePath; + } + + @Override + public boolean equals(Object o) + { + if (o instanceof DataSegmentAndIndexZipFilePath) { + return segment.equals(((DataSegmentAndIndexZipFilePath) o).getSegment()); + } + return false; + } + + @Override + public int hashCode() + { + return Objects.hash(segment.getId(), tmpIndexZipFilePath); + } + + @Override + public String toString() + { + return "DataSegmentAndTmpPath{" + + "segment=" + segment + + ", tmpIndexZipFilePath=" + tmpIndexZipFilePath + + ", finalIndexZipFilePath=" + finalIndexZipFilePath + + '}'; + } +} diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java deleted file mode 100644 index fdc9d28fbed1..000000000000 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndTmpPath.java +++ /dev/null @@ -1,59 +0,0 @@ -package org.apache.druid.indexer; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.timeline.DataSegment; - -import java.util.Objects; - -public class DataSegmentAndTmpPath -{ - private final DataSegment segment; - private final String indexZipFilePath; - - @JsonCreator - public DataSegmentAndTmpPath( - @JsonProperty("segment") DataSegment segment, - @JsonProperty("indexZipFilePath") String indexZipFilePath - ) - { - this.segment = segment; - this.indexZipFilePath = indexZipFilePath; - } - - @JsonProperty - public DataSegment getSegment() - { - return segment; - } - - @JsonProperty - public String getIndexZipFilePath() - { - return indexZipFilePath; - } - - @Override - public boolean equals(Object o) - { - if (o instanceof DataSegmentAndTmpPath) { - return segment.getId().equals(((DataSegmentAndTmpPath) o).getSegment().getId()); - } - return false; - } - - @Override - public int hashCode() - { - return Objects.hash(segment.getId(), indexZipFilePath); - } - - @Override - public String toString() - { - return "DataSegmentAndTmpPath{" + - "segment=" + segment + - ", indexZipFilePath=" + indexZipFilePath + - '}'; - } -} diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java index c5b5684783a8..8b6f5d8b872b 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java @@ -39,7 +39,7 @@ public class HadoopDruidIndexerJob implements Jobby @Nullable private IndexGeneratorJob indexJob; @Nullable - private volatile List publishedSegmentAndTmpPaths = null; + private volatile List publishedSegmentAndTmpPaths = null; @Nullable private String hadoopJobIdFile; @@ -121,7 +121,7 @@ public String getErrorMessage() return indexJob.getErrorMessage(); } - public List getPublishedSegmentAndTmpPaths() + public List getPublishedSegmentAndTmpPaths() { if (publishedSegmentAndTmpPaths == null) { throw new IllegalStateException("Job hasn't run yet. No segments have been published yet."); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 212e971e2adc..ccdcde6b5804 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -102,14 +102,14 @@ public class IndexGeneratorJob implements Jobby { private static final Logger log = new Logger(IndexGeneratorJob.class); - public static List getPublishedSegmentAndTmpPaths(HadoopDruidIndexerConfig config) + public static List getPublishedSegmentAndTmpPaths(HadoopDruidIndexerConfig config) { final Configuration conf = JobHelper.injectSystemProperties(new Configuration(), config); config.addJobProperties(conf); final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.JSON_MAPPER; - ImmutableList.Builder publishedSegmentAndTmpPathsBuilder = ImmutableList.builder(); + ImmutableList.Builder publishedSegmentAndTmpPathsBuilder = ImmutableList.builder(); final Path descriptorInfoDir = config.makeDescriptorInfoDir(); @@ -117,7 +117,7 @@ public static List getPublishedSegmentAndTmpPaths(HadoopD FileSystem fs = descriptorInfoDir.getFileSystem(conf); for (FileStatus status : fs.listStatus(descriptorInfoDir)) { - final DataSegmentAndTmpPath segmentAndTmpPath = jsonMapper.readValue((InputStream) fs.open(status.getPath()), DataSegmentAndTmpPath.class); + final DataSegmentAndIndexZipFilePath segmentAndTmpPath = jsonMapper.readValue((InputStream) fs.open(status.getPath()), DataSegmentAndIndexZipFilePath.class); publishedSegmentAndTmpPathsBuilder.add(segmentAndTmpPath); log.info("Adding segment %s to the list of published segments", segmentAndTmpPath.getSegment().getId()); } @@ -133,7 +133,7 @@ public static List getPublishedSegmentAndTmpPaths(HadoopD catch (IOException e) { throw new RuntimeException(e); } - List publishedSegmentAndTmpPaths = publishedSegmentAndTmpPathsBuilder.build(); + List publishedSegmentAndTmpPaths = publishedSegmentAndTmpPathsBuilder.build(); return publishedSegmentAndTmpPaths; } @@ -809,7 +809,7 @@ public void doRun() 0 ); - final DataSegmentAndTmpPath segmentAndTmpPath = JobHelper.serializeOutIndex( + final DataSegmentAndIndexZipFilePath segmentAndTmpPath = JobHelper.serializeOutIndex( segmentTemplate, context.getConfiguration(), context, diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index cad99d3528a1..08edd2e988a4 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -436,7 +436,7 @@ public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) return succeeded; } - public static DataSegmentAndTmpPath serializeOutIndex( + public static DataSegmentAndIndexZipFilePath serializeOutIndex( final DataSegment segmentTemplate, final Configuration configuration, final Progressable progressable, @@ -485,12 +485,16 @@ public long push() throws IOException .withSize(size.get()) .withBinaryVersion(SegmentUtils.getVersionFromDir(mergedBase)); - return new DataSegmentAndTmpPath(finalSegment, tmpPath.toUri().getPath()); + return new DataSegmentAndIndexZipFilePath( + finalSegment, + tmpPath.toUri().getPath(), + finalIndexZipFilePath.toUri().getPath() + ); } public static void writeSegmentDescriptor( final FileSystem outputFS, - final DataSegmentAndTmpPath segmentAndPath, + final DataSegmentAndIndexZipFilePath segmentAndPath, final Path descriptorPath, final Progressable progressable ) @@ -771,7 +775,7 @@ public static void renameIndexFilesForSegments( HadoopIngestionSpec indexerSchema, String segmentOutputPath, String workingPath, - List segmentsAndTmpPaths + List segmentsAndIndexZipFilePath ) throws IOException { log.info("Building HadoopDruidIndexerConfig"); @@ -785,11 +789,11 @@ public static void renameIndexFilesForSegments( config.addJobProperties(configuration); JobHelper.injectDruidProperties(configuration, config); log.info("Built Configuration"); - for (DataSegmentAndTmpPath segmentAndTmpPath : segmentsAndTmpPaths) { - log.info("tmpPath: [%s]", segmentAndTmpPath.getIndexZipFilePath()); + for (DataSegmentAndIndexZipFilePath segmentAndTmpPath : segmentsAndIndexZipFilePath) { + log.info("tmpPath: [%s]", segmentAndTmpPath.getTmpIndexZipFilePath()); log.info("segmentId: [%s]", segmentAndTmpPath.getSegment().getId()); - Path tmpPath = new Path(segmentAndTmpPath.getIndexZipFilePath()); - Path finalIndexZipFilePath = new Path(tmpPath.getParent(), INDEX_ZIP); + Path tmpPath = new Path(segmentAndTmpPath.getTmpIndexZipFilePath()); + Path finalIndexZipFilePath = new Path(segmentAndTmpPath.getFinalIndexZipFilePath()); final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); log.info("about to rename segment index file"); if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java index a2781faa9282..cfa8aa4bb12c 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java @@ -43,7 +43,7 @@ public MetadataStorageUpdaterJob( @Override public boolean run() { - final List segmentAndTmpPaths = IndexGeneratorJob.getPublishedSegmentAndTmpPaths(config); + final List segmentAndTmpPaths = IndexGeneratorJob.getPublishedSegmentAndTmpPaths(config); final List segments = segmentAndTmpPaths.stream().map(s -> s.getSegment()).collect(Collectors.toList()); final String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable(); handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.JSON_MAPPER); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 69f0e0604e71..b9f5347733d7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -31,7 +31,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; -import org.apache.druid.indexer.DataSegmentAndTmpPath; +import org.apache.druid.indexer.DataSegmentAndIndexZipFilePath; import org.apache.druid.indexer.HadoopDruidDetermineConfigurationJob; import org.apache.druid.indexer.HadoopDruidIndexerConfig; import org.apache.druid.indexer.HadoopDruidIndexerJob; @@ -91,8 +91,6 @@ public class HadoopIndexTask extends HadoopTask implements ChatHandler private static final Logger log = new Logger(HadoopIndexTask.class); private static final String HADOOP_JOB_ID_FILENAME = "mapReduceJobId.json"; private static final String TYPE = "index_hadoop"; - private static final int NUM_RETRIES = 8; - public static final String INDEX_ZIP = "index.zip"; private TaskConfig taskConfig = null; private static String getTheDataSource(HadoopIngestionSpec spec) @@ -453,9 +451,9 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception ); log.info("about to get segment files"); - List dataSegmentAndTmpPaths = buildSegmentsStatus.getDataSegmentAndTmpPaths(); + List dataSegmentAndIndexZipFilePaths = buildSegmentsStatus.getDataSegmentAndIndexZipFilePaths(); log.info("about to rename segment files"); - if (dataSegmentAndTmpPaths != null) { + if (dataSegmentAndIndexZipFilePaths != null) { log.info("found non-null segment files"); try { @@ -464,7 +462,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception toolbox.getJsonMapper().writeValueAsString(indexerSchema), segmentOutputPath, workingPath, - toolbox.getJsonMapper().writeValueAsString(dataSegmentAndTmpPaths) + toolbox.getJsonMapper().writeValueAsString(dataSegmentAndIndexZipFilePaths) ); } finally { @@ -472,9 +470,9 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception } ingestionState = IngestionState.COMPLETED; - toolbox.publishSegments(dataSegmentAndTmpPaths.stream() - .map(DataSegmentAndTmpPath::getSegment) - .collect(Collectors.toList())); + toolbox.publishSegments(dataSegmentAndIndexZipFilePaths.stream() + .map(DataSegmentAndIndexZipFilePath::getSegment) + .collect(Collectors.toList())); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } else { @@ -539,7 +537,7 @@ private void renameSegmentIndexFilesJob( String hadoopIngestionSpecStr, String segmentOutputPath, String workingPath, - String dataSegmentAndTmpPathListStr + String dataSegmentAndIndexZipFilePathListStr ) { log.info("In renameSegmentIndexFilesJob"); @@ -559,15 +557,15 @@ private void renameSegmentIndexFilesJob( hadoopIngestionSpecStr, workingPath, segmentOutputPath, - dataSegmentAndTmpPathListStr + dataSegmentAndIndexZipFilePathListStr }; log.info( - "hadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndTmpPathListStr: [%s]", + "hadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndIndexZipFilePathListStr: [%s]", hadoopIngestionSpecStr, workingPath, segmentOutputPath, - dataSegmentAndTmpPathListStr + dataSegmentAndIndexZipFilePathListStr ); Class buildKillJobRunnerClass = renameSegmentIndexFilesRunner.getClass(); Method renameSegmentIndexFiles = buildKillJobRunnerClass.getMethod( @@ -868,8 +866,8 @@ public String[] runTask(String[] args) throws Exception @SuppressWarnings("unused") public static class HadoopRenameSegmentIndexFilesRunner { - TypeReference> LIST_DATA_SEGMENT_AND_TMP_PATH = - new TypeReference>() + TypeReference> LIST_DATA_SEGMENT_AND_TMP_PATH = + new TypeReference>() { }; @@ -881,24 +879,24 @@ public void runTask(String[] args) throws Exception String hadoopIngestionSpecStr = args[0]; String workingPath = args[1]; String segmentOutputPath = args[2]; - String dataSegmentAndTmpPathListStr = args[3]; + String dataSegmentAndIndexZipFilePathListStr = args[3]; log.info( - "HadoopRenameSegmentIndexFilesRunner: HadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndTmpPathListStr: [%s]", + "HadoopRenameSegmentIndexFilesRunner: HadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndIndexZipFilePathListStr: [%s]", hadoopIngestionSpecStr, workingPath, segmentOutputPath, - dataSegmentAndTmpPathListStr + dataSegmentAndIndexZipFilePathListStr ); HadoopIngestionSpec indexerSchema; - List dataSegmentAndTmpPaths; + List dataSegmentAndIndexZipFilePaths; try { indexerSchema = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( hadoopIngestionSpecStr, HadoopIngestionSpec.class ); - dataSegmentAndTmpPaths = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( - dataSegmentAndTmpPathListStr, + dataSegmentAndIndexZipFilePaths = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( + dataSegmentAndIndexZipFilePathListStr, LIST_DATA_SEGMENT_AND_TMP_PATH ); } @@ -909,32 +907,34 @@ public void runTask(String[] args) throws Exception ); throw e; } - JobHelper.renameIndexFilesForSegments(indexerSchema, segmentOutputPath, workingPath, dataSegmentAndTmpPaths); + JobHelper.renameIndexFilesForSegments(indexerSchema, segmentOutputPath, workingPath, + dataSegmentAndIndexZipFilePaths + ); } } public static class HadoopIndexGeneratorInnerProcessingStatus { - private final List dataSegmentAndTmpPaths; + private final List dataSegmentAndIndexZipFilePaths; private final Map metrics; private final String errorMsg; @JsonCreator public HadoopIndexGeneratorInnerProcessingStatus( - @JsonProperty("dataSegmentAndTmpPaths") List dataSegmentAndTmpPaths, + @JsonProperty("dataSegmentAndIndexZipFilePaths") List dataSegmentAndIndexZipFilePaths, @JsonProperty("metrics") Map metrics, @JsonProperty("errorMsg") String errorMsg ) { - this.dataSegmentAndTmpPaths = dataSegmentAndTmpPaths; + this.dataSegmentAndIndexZipFilePaths = dataSegmentAndIndexZipFilePaths; this.metrics = metrics; this.errorMsg = errorMsg; } @JsonProperty - public List getDataSegmentAndTmpPaths() + public List getDataSegmentAndIndexZipFilePaths() { - return dataSegmentAndTmpPaths; + return dataSegmentAndIndexZipFilePaths; } @JsonProperty From 9f5bc9470f96e8b49a1224e7676237f0cd64ac12 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 21:28:52 -0400 Subject: [PATCH 06/20] * cleanup --- .../DataSegmentAndIndexZipFilePath.java | 2 +- .../druid/indexer/HadoopDruidIndexerJob.java | 10 +- .../druid/indexer/IndexGeneratorJob.java | 20 +- .../org/apache/druid/indexer/JobHelper.java | 214 +++++++++--------- .../indexer/MetadataStorageUpdaterJob.java | 4 +- .../druid/indexer/IndexGeneratorJobTest.java | 6 +- .../indexing/common/task/HadoopIndexTask.java | 21 +- 7 files changed, 123 insertions(+), 154 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java index f8b764863ea2..0b1a5a48a58a 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java @@ -60,7 +60,7 @@ public int hashCode() @Override public String toString() { - return "DataSegmentAndTmpPath{" + + return "DataSegmentAndIndexZipFilePath{" + "segment=" + segment + ", tmpIndexZipFilePath=" + tmpIndexZipFilePath + ", finalIndexZipFilePath=" + finalIndexZipFilePath + diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java index 8b6f5d8b872b..f5a22782e02d 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java @@ -39,7 +39,7 @@ public class HadoopDruidIndexerJob implements Jobby @Nullable private IndexGeneratorJob indexJob; @Nullable - private volatile List publishedSegmentAndTmpPaths = null; + private volatile List publishedSegmentAndIndexZipFilePaths = null; @Nullable private String hadoopJobIdFile; @@ -90,7 +90,7 @@ public boolean run() @Override public boolean run() { - publishedSegmentAndTmpPaths = IndexGeneratorJob.getPublishedSegmentAndTmpPaths(config); + publishedSegmentAndIndexZipFilePaths = IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(config); return true; } } @@ -121,12 +121,12 @@ public String getErrorMessage() return indexJob.getErrorMessage(); } - public List getPublishedSegmentAndTmpPaths() + public List getPublishedSegmentAndIndexZipFilePaths() { - if (publishedSegmentAndTmpPaths == null) { + if (publishedSegmentAndIndexZipFilePaths == null) { throw new IllegalStateException("Job hasn't run yet. No segments have been published yet."); } - return publishedSegmentAndTmpPaths; + return publishedSegmentAndIndexZipFilePaths; } public void setHadoopJobIdFile(String hadoopJobIdFile) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index ccdcde6b5804..9124b9b4c1d8 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -102,14 +102,14 @@ public class IndexGeneratorJob implements Jobby { private static final Logger log = new Logger(IndexGeneratorJob.class); - public static List getPublishedSegmentAndTmpPaths(HadoopDruidIndexerConfig config) + public static List getPublishedSegmentAndIndexZipFilePaths(HadoopDruidIndexerConfig config) { final Configuration conf = JobHelper.injectSystemProperties(new Configuration(), config); config.addJobProperties(conf); final ObjectMapper jsonMapper = HadoopDruidIndexerConfig.JSON_MAPPER; - ImmutableList.Builder publishedSegmentAndTmpPathsBuilder = ImmutableList.builder(); + ImmutableList.Builder publishedSegmentAndIndexZipFilePathsBuilder = ImmutableList.builder(); final Path descriptorInfoDir = config.makeDescriptorInfoDir(); @@ -117,9 +117,9 @@ public static List getPublishedSegmentAndTmpPath FileSystem fs = descriptorInfoDir.getFileSystem(conf); for (FileStatus status : fs.listStatus(descriptorInfoDir)) { - final DataSegmentAndIndexZipFilePath segmentAndTmpPath = jsonMapper.readValue((InputStream) fs.open(status.getPath()), DataSegmentAndIndexZipFilePath.class); - publishedSegmentAndTmpPathsBuilder.add(segmentAndTmpPath); - log.info("Adding segment %s to the list of published segments", segmentAndTmpPath.getSegment().getId()); + final DataSegmentAndIndexZipFilePath segmentAndIndexZipFilePath = jsonMapper.readValue((InputStream) fs.open(status.getPath()), DataSegmentAndIndexZipFilePath.class); + publishedSegmentAndIndexZipFilePathsBuilder.add(segmentAndIndexZipFilePath); + log.info("Adding segment %s to the list of published segments", segmentAndIndexZipFilePath.getSegment().getId()); } } catch (FileNotFoundException e) { @@ -133,9 +133,9 @@ public static List getPublishedSegmentAndTmpPath catch (IOException e) { throw new RuntimeException(e); } - List publishedSegmentAndTmpPaths = publishedSegmentAndTmpPathsBuilder.build(); + List publishedSegmentAndIndexZipFilePaths = publishedSegmentAndIndexZipFilePathsBuilder.build(); - return publishedSegmentAndTmpPaths; + return publishedSegmentAndIndexZipFilePaths; } private final HadoopDruidIndexerConfig config; @@ -809,7 +809,7 @@ public void doRun() 0 ); - final DataSegmentAndIndexZipFilePath segmentAndTmpPath = JobHelper.serializeOutIndex( + final DataSegmentAndIndexZipFilePath segmentAndIndexZipFilePath = JobHelper.serializeOutIndex( segmentTemplate, context.getConfiguration(), context, @@ -831,7 +831,7 @@ public void doRun() HadoopDruidIndexerConfig.DATA_SEGMENT_PUSHER ); - Path descriptorPath = config.makeDescriptorInfoPath(segmentAndTmpPath.getSegment()); + Path descriptorPath = config.makeDescriptorInfoPath(segmentAndIndexZipFilePath.getSegment()); descriptorPath = JobHelper.prependFSIfNullScheme( FileSystem.get( descriptorPath.toUri(), @@ -842,7 +842,7 @@ public void doRun() log.info("Writing descriptor to path[%s]", descriptorPath); JobHelper.writeSegmentDescriptor( config.makeDescriptorInfoDir().getFileSystem(context.getConfiguration()), - segmentAndTmpPath, + segmentAndIndexZipFilePath, descriptorPath, context ); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index 08edd2e988a4..59bbb14e571b 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -447,9 +447,6 @@ public static DataSegmentAndIndexZipFilePath serializeOutIndex( ) throws IOException { - log.info("in Reducer: finalIndexZipFilePath: [%s], tmpPath: [%s]", - finalIndexZipFilePath.toUri().getPath(), - tmpPath.toUri().getPath()); final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); final AtomicLong size = new AtomicLong(0L); final DataPusher zipPusher = (DataPusher) RetryProxy.create( @@ -630,6 +627,107 @@ public static Path makeTmpPath( ); } + /** + * Renames the index files for the segments. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename + * which will not overwrite + */ + public static void renameIndexFilesForSegments( + HadoopIngestionSpec indexerSchema, + String segmentOutputPath, + String workingPath, + List segmentsAndIndexZipFilePath + ) throws IOException + { + HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec( + indexerSchema + .withIOConfig(indexerSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) + .withTuningConfig(indexerSchema.getTuningConfig().withWorkingPath(workingPath)) + ); + final Configuration configuration = JobHelper.injectSystemProperties(new Configuration(), config); + config.addJobProperties(configuration); + JobHelper.injectDruidProperties(configuration, config); + for (DataSegmentAndIndexZipFilePath segmentAndIndexZipFilePath : segmentsAndIndexZipFilePath) { + Path tmpPath = new Path(segmentAndIndexZipFilePath.getTmpIndexZipFilePath()); + Path finalIndexZipFilePath = new Path(segmentAndIndexZipFilePath.getFinalIndexZipFilePath()); + final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); + if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { + throw new IOE( + "Unable to rename [%s] to [%s]", + tmpPath.toUri().toString(), + finalIndexZipFilePath.toUri().toString() + ); + } + } + } + + /** + * Rename the file. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename + * which will not overwrite + * + * @param outputFS The output fs + * @param indexZipFilePath The original file path + * @param finalIndexZipFilePath The to rename the original file to + * + * @return False if a rename failed, true otherwise (rename success or no rename needed) + */ + public static boolean renameIndexFile( + final FileSystem outputFS, + final Path indexZipFilePath, + final Path finalIndexZipFilePath + ) + { + try { + return RetryUtils.retry( + () -> { + final boolean needRename; + + if (outputFS.exists(finalIndexZipFilePath)) { + // NativeS3FileSystem.rename won't overwrite, so we might need to delete the old index first + final FileStatus zipFile = outputFS.getFileStatus(indexZipFilePath); + final FileStatus finalIndexZipFile = outputFS.getFileStatus(finalIndexZipFilePath); + + if (zipFile.getModificationTime() >= finalIndexZipFile.getModificationTime() + || zipFile.getLen() != finalIndexZipFile.getLen()) { + log.info( + "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", + finalIndexZipFile.getPath(), + DateTimes.utc(finalIndexZipFile.getModificationTime()), + finalIndexZipFile.getLen(), + zipFile.getPath(), + DateTimes.utc(zipFile.getModificationTime()), + zipFile.getLen() + ); + outputFS.delete(finalIndexZipFilePath, false); + needRename = true; + } else { + log.info( + "File[%s / %s / %sB] existed and will be kept", + finalIndexZipFile.getPath(), + DateTimes.utc(finalIndexZipFile.getModificationTime()), + finalIndexZipFile.getLen() + ); + needRename = false; + } + } else { + needRename = true; + } + + if (needRename) { + log.info("Attempting rename from [%s] to [%s]", indexZipFilePath, finalIndexZipFilePath); + return outputFS.rename(indexZipFilePath, finalIndexZipFilePath); + } else { + return true; + } + }, + FileUtils.IS_EXCEPTION, + NUM_RETRIES + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + public static Path prependFSIfNullScheme(FileSystem fs, Path path) { @@ -766,114 +864,4 @@ public static String getJobTrackerAddress(Configuration config) } return jobTrackerAddress; } - - /** - * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename - * which will not overwrite - */ - public static void renameIndexFilesForSegments( - HadoopIngestionSpec indexerSchema, - String segmentOutputPath, - String workingPath, - List segmentsAndIndexZipFilePath - ) throws IOException - { - log.info("Building HadoopDruidIndexerConfig"); - HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec( - indexerSchema - .withIOConfig(indexerSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) - .withTuningConfig(indexerSchema.getTuningConfig().withWorkingPath(workingPath)) - ); - log.info("Building Configuration"); - final Configuration configuration = JobHelper.injectSystemProperties(new Configuration(), config); - config.addJobProperties(configuration); - JobHelper.injectDruidProperties(configuration, config); - log.info("Built Configuration"); - for (DataSegmentAndIndexZipFilePath segmentAndTmpPath : segmentsAndIndexZipFilePath) { - log.info("tmpPath: [%s]", segmentAndTmpPath.getTmpIndexZipFilePath()); - log.info("segmentId: [%s]", segmentAndTmpPath.getSegment().getId()); - Path tmpPath = new Path(segmentAndTmpPath.getTmpIndexZipFilePath()); - Path finalIndexZipFilePath = new Path(segmentAndTmpPath.getFinalIndexZipFilePath()); - final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); - log.info("about to rename segment index file"); - if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { - throw new IOE( - "Unable to rename [%s] to [%s]", - tmpPath.toUri().toString(), - finalIndexZipFilePath.toUri().toString() - ); - } - } - } - - /** - * Rename the files. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename - * which will not overwrite - * - * @param outputFS The output fs - * @param indexZipFilePath The original file path - * @param finalIndexZipFilePath The to rename the original file to - * - * @return False if a rename failed, true otherwise (rename success or no rename needed) - */ - public static boolean renameIndexFile( - final FileSystem outputFS, - final Path indexZipFilePath, - final Path finalIndexZipFilePath - ) - { - log.info("renameIndexFile: finalIndexZipFilePath: [%s], tmpPath: [%s]", - finalIndexZipFilePath.toUri().getPath(), - indexZipFilePath.toUri().getPath()); - try { - return RetryUtils.retry( - () -> { - final boolean needRename; - - if (outputFS.exists(finalIndexZipFilePath)) { - // NativeS3FileSystem.rename won't overwrite, so we might need to delete the old index first - final FileStatus zipFile = outputFS.getFileStatus(indexZipFilePath); - final FileStatus finalIndexZipFile = outputFS.getFileStatus(finalIndexZipFilePath); - - if (zipFile.getModificationTime() >= finalIndexZipFile.getModificationTime() - || zipFile.getLen() != finalIndexZipFile.getLen()) { - log.info( - "File[%s / %s / %sB] existed, but wasn't the same as [%s / %s / %sB]", - finalIndexZipFile.getPath(), - DateTimes.utc(finalIndexZipFile.getModificationTime()), - finalIndexZipFile.getLen(), - zipFile.getPath(), - DateTimes.utc(zipFile.getModificationTime()), - zipFile.getLen() - ); - outputFS.delete(finalIndexZipFilePath, false); - needRename = true; - } else { - log.info( - "File[%s / %s / %sB] existed and will be kept", - finalIndexZipFile.getPath(), - DateTimes.utc(finalIndexZipFile.getModificationTime()), - finalIndexZipFile.getLen() - ); - needRename = false; - } - } else { - needRename = true; - } - - if (needRename) { - log.info("Attempting rename from [%s] to [%s]", indexZipFilePath, finalIndexZipFilePath); - return outputFS.rename(indexZipFilePath, finalIndexZipFilePath); - } else { - return true; - } - }, - FileUtils.IS_EXCEPTION, - NUM_RETRIES - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } } diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java index cfa8aa4bb12c..bdadc95010e3 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/MetadataStorageUpdaterJob.java @@ -43,8 +43,8 @@ public MetadataStorageUpdaterJob( @Override public boolean run() { - final List segmentAndTmpPaths = IndexGeneratorJob.getPublishedSegmentAndTmpPaths(config); - final List segments = segmentAndTmpPaths.stream().map(s -> s.getSegment()).collect(Collectors.toList()); + final List segmentAndIndexZipFilePaths = IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(config); + final List segments = segmentAndIndexZipFilePaths.stream().map(s -> s.getSegment()).collect(Collectors.toList()); final String segmentTable = config.getSchema().getIOConfig().getMetadataUpdateSpec().getSegmentTable(); handler.publishSegments(segmentTable, segments, HadoopDruidIndexerConfig.JSON_MAPPER); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index 067b3077bc9e..3b4e6cb5d0f3 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -624,9 +624,9 @@ private void verifyJob(IndexGeneratorJob job) throws IOException final Map> intervalToSegments = new HashMap<>(); IndexGeneratorJob - .getPublishedSegmentAndTmpPaths(config) - .forEach(segmentAndTmpPath -> intervalToSegments.computeIfAbsent(segmentAndTmpPath.getSegment().getInterval(), k -> new ArrayList<>()) - .add(segmentAndTmpPath.getSegment())); + .getPublishedSegmentAndIndexZipFilePaths(config) + .forEach(segmentAndIndexZipFilePath -> intervalToSegments.computeIfAbsent(segmentAndIndexZipFilePath.getSegment().getInterval(), k -> new ArrayList<>()) + .add(segmentAndIndexZipFilePath.getSegment())); final Map> intervalToIndexFiles = new HashMap<>(); int segmentNum = 0; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index b9f5347733d7..fc28d4161d5b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -342,9 +342,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception "runTask", determinePartitionsInput.getClass() ); - - HadoopDruidIndexerConfig config; - try { Thread.currentThread().setContextClassLoader(loader); @@ -371,7 +368,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception errorMsg ); } - log.info("workingPath: [%s], segmentOutputPath: [%s]", workingPath, segmentOutputPath); } catch (Exception e) { throw new RuntimeException(e); @@ -438,7 +434,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception try { Thread.currentThread().setContextClassLoader(loader); - log.info("about to invoke job"); ingestionState = IngestionState.BUILD_SEGMENTS; final String jobStatusString = (String) innerProcessingRunTask.invoke( innerProcessingRunner, @@ -450,12 +445,8 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception HadoopIndexGeneratorInnerProcessingStatus.class ); - log.info("about to get segment files"); List dataSegmentAndIndexZipFilePaths = buildSegmentsStatus.getDataSegmentAndIndexZipFilePaths(); - log.info("about to rename segment files"); if (dataSegmentAndIndexZipFilePaths != null) { - log.info("found non-null segment files"); - try { Thread.currentThread().setContextClassLoader(oldLoader); renameSegmentIndexFilesJob( @@ -476,7 +467,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.success(getId()); } else { - log.info("found null segment files :("); errorMsg = buildSegmentsStatus.getErrorMsg(); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); return TaskStatus.failure( @@ -540,7 +530,6 @@ private void renameSegmentIndexFilesJob( String dataSegmentAndIndexZipFilePathListStr ) { - log.info("In renameSegmentIndexFilesJob"); final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); try { ClassLoader loader = HadoopTask.buildClassLoader( @@ -560,13 +549,6 @@ private void renameSegmentIndexFilesJob( dataSegmentAndIndexZipFilePathListStr }; - log.info( - "hadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndIndexZipFilePathListStr: [%s]", - hadoopIngestionSpecStr, - workingPath, - segmentOutputPath, - dataSegmentAndIndexZipFilePathListStr - ); Class buildKillJobRunnerClass = renameSegmentIndexFilesRunner.getClass(); Method renameSegmentIndexFiles = buildKillJobRunnerClass.getMethod( "runTask", @@ -792,10 +774,9 @@ public String runTask(String[] args) throws Exception log.info("Starting a hadoop index generator job..."); try { if (job.run()) { - log.info("Constructing HadoopIndexGeneratorInnerProcessingStatus with segmentsAndPaths..."); return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( new HadoopIndexGeneratorInnerProcessingStatus( - job.getPublishedSegmentAndTmpPaths(), + job.getPublishedSegmentAndIndexZipFilePaths(), job.getStats(), null ) From 2f715238ac0973b6d91a9e684e07daa04bdbc1bc Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 22:00:22 -0400 Subject: [PATCH 07/20] * more cleanup --- .../org/apache/druid/indexer/JobHelper.java | 8 +-- .../indexing/common/task/HadoopIndexTask.java | 50 +++---------------- 2 files changed, 8 insertions(+), 50 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index 59bbb14e571b..1c3d6609fa58 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -633,16 +633,10 @@ public static Path makeTmpPath( */ public static void renameIndexFilesForSegments( HadoopIngestionSpec indexerSchema, - String segmentOutputPath, - String workingPath, List segmentsAndIndexZipFilePath ) throws IOException { - HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec( - indexerSchema - .withIOConfig(indexerSchema.getIOConfig().withSegmentOutputPath(segmentOutputPath)) - .withTuningConfig(indexerSchema.getTuningConfig().withWorkingPath(workingPath)) - ); + HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec(indexerSchema); final Configuration configuration = JobHelper.injectSystemProperties(new Configuration(), config); config.addJobProperties(configuration); JobHelper.injectDruidProperties(configuration, config); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index fc28d4161d5b..118573624e08 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -334,8 +334,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception }; HadoopIngestionSpec indexerSchema; - String workingPath; - String segmentOutputPath; final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); Class determinePartitionsRunnerClass = determinePartitionsInnerProcessingRunner.getClass(); Method determinePartitionsInnerProcessingRunTask = determinePartitionsRunnerClass.getMethod( @@ -358,8 +356,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); indexerSchema = determineConfigStatus.getSchema(); - workingPath = determineConfigStatus.getWorkingPath(); - segmentOutputPath = determineConfigStatus.getSegmentOutputPath(); if (indexerSchema == null) { errorMsg = determineConfigStatus.getErrorMsg(); toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); @@ -451,8 +447,6 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception Thread.currentThread().setContextClassLoader(oldLoader); renameSegmentIndexFilesJob( toolbox.getJsonMapper().writeValueAsString(indexerSchema), - segmentOutputPath, - workingPath, toolbox.getJsonMapper().writeValueAsString(dataSegmentAndIndexZipFilePaths) ); } @@ -525,8 +519,6 @@ private void killHadoopJob() private void renameSegmentIndexFilesJob( String hadoopIngestionSpecStr, - String segmentOutputPath, - String workingPath, String dataSegmentAndIndexZipFilePathListStr ) { @@ -544,8 +536,6 @@ private void renameSegmentIndexFilesJob( String[] renameSegmentIndexFilesJobInput = new String[]{ hadoopIngestionSpecStr, - workingPath, - segmentOutputPath, dataSegmentAndIndexZipFilePathListStr }; @@ -719,11 +709,11 @@ public String runTask(String[] args) throws Exception log.info("Starting a hadoop determine configuration job..."); if (job.run()) { return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( - new HadoopDetermineConfigInnerProcessingStatus(config.getSchema(), workingPath, segmentOutputPath, job.getStats(), null) + new HadoopDetermineConfigInnerProcessingStatus(config.getSchema(), job.getStats(), null) ); } else { return HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString( - new HadoopDetermineConfigInnerProcessingStatus(null, null, null, job.getStats(), job.getErrorMessage()) + new HadoopDetermineConfigInnerProcessingStatus(null, job.getStats(), job.getErrorMessage()) ); } } @@ -854,20 +844,11 @@ public static class HadoopRenameSegmentIndexFilesRunner public void runTask(String[] args) throws Exception { - if (args.length != 4) { + if (args.length != 2) { log.warn("HadoopRenameSegmentIndexFilesRunner called with improper number of arguments"); } String hadoopIngestionSpecStr = args[0]; - String workingPath = args[1]; - String segmentOutputPath = args[2]; - String dataSegmentAndIndexZipFilePathListStr = args[3]; - log.info( - "HadoopRenameSegmentIndexFilesRunner: HadoopIngestionSpecStr: [%s], workingPath: [%s], segmentOutputPath: [%s], dataSegmentAndIndexZipFilePathListStr: [%s]", - hadoopIngestionSpecStr, - workingPath, - segmentOutputPath, - dataSegmentAndIndexZipFilePathListStr - ); + String dataSegmentAndIndexZipFilePathListStr = args[1]; HadoopIngestionSpec indexerSchema; List dataSegmentAndIndexZipFilePaths; @@ -888,8 +869,9 @@ public void runTask(String[] args) throws Exception ); throw e; } - JobHelper.renameIndexFilesForSegments(indexerSchema, segmentOutputPath, workingPath, - dataSegmentAndIndexZipFilePaths + JobHelper.renameIndexFilesForSegments( + indexerSchema, + dataSegmentAndIndexZipFilePaths ); } } @@ -934,23 +916,17 @@ public String getErrorMsg() public static class HadoopDetermineConfigInnerProcessingStatus { private final HadoopIngestionSpec schema; - private final String workingPath; - private final String segmentOutputPath; private final Map metrics; private final String errorMsg; @JsonCreator public HadoopDetermineConfigInnerProcessingStatus( @JsonProperty("schema") HadoopIngestionSpec schema, - @JsonProperty("workingPath") String workingPath, - @JsonProperty("segmentOutputPath") String segmentOutputPath, @JsonProperty("metrics") Map metrics, @JsonProperty("errorMsg") String errorMsg ) { this.schema = schema; - this.workingPath = workingPath; - this.segmentOutputPath = segmentOutputPath; this.metrics = metrics; this.errorMsg = errorMsg; } @@ -961,18 +937,6 @@ public HadoopIngestionSpec getSchema() return schema; } - @JsonProperty - public String getWorkingPath() - { - return workingPath; - } - - @JsonProperty - public String getSegmentOutputPath() - { - return segmentOutputPath; - } - @JsonProperty public Map getMetrics() { From 22061c3be353cd3c8bbd1c9db98095b213a52a6e Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Wed, 7 Apr 2021 22:30:49 -0400 Subject: [PATCH 08/20] * more cleanup --- .../apache/druid/indexer/HadoopDruidIndexerConfig.java | 2 +- .../druid/indexing/common/task/HadoopIndexTask.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java index cf6f34220c6a..54c8f073a9b9 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerConfig.java @@ -578,7 +578,7 @@ void addJobProperties(Job job) addJobProperties(job.getConfiguration()); } - public void addJobProperties(Configuration conf) + void addJobProperties(Configuration conf) { for (final Map.Entry entry : schema.getTuningConfig().getJobProperties().entrySet()) { conf.set(entry.getKey(), entry.getValue()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 118573624e08..8acf6756828d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -539,8 +539,8 @@ private void renameSegmentIndexFilesJob( dataSegmentAndIndexZipFilePathListStr }; - Class buildKillJobRunnerClass = renameSegmentIndexFilesRunner.getClass(); - Method renameSegmentIndexFiles = buildKillJobRunnerClass.getMethod( + Class buildRenameSegmentIndexFilesJobRunnerClass = renameSegmentIndexFilesRunner.getClass(); + Method renameSegmentIndexFiles = buildRenameSegmentIndexFilesJobRunnerClass.getMethod( "runTask", renameSegmentIndexFilesJobInput.getClass() ); @@ -837,7 +837,7 @@ public String[] runTask(String[] args) throws Exception @SuppressWarnings("unused") public static class HadoopRenameSegmentIndexFilesRunner { - TypeReference> LIST_DATA_SEGMENT_AND_TMP_PATH = + TypeReference> LIST_DATA_SEGMENT_AND_INDEX_ZIP_FILE_PATH = new TypeReference>() { }; @@ -859,7 +859,7 @@ public void runTask(String[] args) throws Exception ); dataSegmentAndIndexZipFilePaths = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( dataSegmentAndIndexZipFilePathListStr, - LIST_DATA_SEGMENT_AND_TMP_PATH + LIST_DATA_SEGMENT_AND_INDEX_ZIP_FILE_PATH ); } catch (Exception e) { From 44f07073ef9d8b8f7ec6996b487db2ee21d057db Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Thu, 8 Apr 2021 00:29:14 -0400 Subject: [PATCH 09/20] * add license header --- .../DataSegmentAndIndexZipFilePath.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java index 0b1a5a48a58a..e1a6aea80790 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java @@ -1,3 +1,22 @@ +/* + * 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.druid.indexer; import com.fasterxml.jackson.annotation.JsonCreator; From ce168cf14422611b7f675e3439545db7af58a2db Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Thu, 8 Apr 2021 15:11:46 -0400 Subject: [PATCH 10/20] * Add unit tests --- indexing-hadoop/pom.xml | 15 ++ .../DataSegmentAndIndexZipFilePath.java | 5 +- .../org/apache/druid/indexer/JobHelper.java | 6 +- .../DataSegmentAndIndexZipFilePathTest.java | 185 ++++++++++++++++++ .../apache/druid/indexer/JobHelperTest.java | 8 + .../MetadataStorageUpdaterJobTest.java | 82 ++++++++ 6 files changed, 297 insertions(+), 4 deletions(-) create mode 100644 indexing-hadoop/src/test/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePathTest.java create mode 100644 indexing-hadoop/src/test/java/org/apache/druid/indexer/MetadataStorageUpdaterJobTest.java diff --git a/indexing-hadoop/pom.xml b/indexing-hadoop/pom.xml index 9557ab5b759b..8eacc7e2220a 100644 --- a/indexing-hadoop/pom.xml +++ b/indexing-hadoop/pom.xml @@ -202,6 +202,21 @@ mockito-core test + + org.powermock + powermock-core + test + + + org.powermock + powermock-module-junit4 + test + + + org.powermock + powermock-api-easymock + test + diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java index e1a6aea80790..e6a7117ea365 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java @@ -65,7 +65,10 @@ public String getFinalIndexZipFilePath() public boolean equals(Object o) { if (o instanceof DataSegmentAndIndexZipFilePath) { - return segment.equals(((DataSegmentAndIndexZipFilePath) o).getSegment()); + DataSegmentAndIndexZipFilePath that = (DataSegmentAndIndexZipFilePath) o; + return segment.equals(((DataSegmentAndIndexZipFilePath) o).getSegment()) + && tmpIndexZipFilePath.equals(that.getTmpIndexZipFilePath()) + && finalIndexZipFilePath.equals(that.getFinalIndexZipFilePath()); } return false; } diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index 1c3d6609fa58..67f8ce75cff6 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -633,14 +633,14 @@ public static Path makeTmpPath( */ public static void renameIndexFilesForSegments( HadoopIngestionSpec indexerSchema, - List segmentsAndIndexZipFilePath + List segmentAndIndexZipFilePaths ) throws IOException { HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec(indexerSchema); final Configuration configuration = JobHelper.injectSystemProperties(new Configuration(), config); config.addJobProperties(configuration); JobHelper.injectDruidProperties(configuration, config); - for (DataSegmentAndIndexZipFilePath segmentAndIndexZipFilePath : segmentsAndIndexZipFilePath) { + for (DataSegmentAndIndexZipFilePath segmentAndIndexZipFilePath : segmentAndIndexZipFilePaths) { Path tmpPath = new Path(segmentAndIndexZipFilePath.getTmpIndexZipFilePath()); Path finalIndexZipFilePath = new Path(segmentAndIndexZipFilePath.getFinalIndexZipFilePath()); final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); @@ -664,7 +664,7 @@ public static void renameIndexFilesForSegments( * * @return False if a rename failed, true otherwise (rename success or no rename needed) */ - public static boolean renameIndexFile( + private static boolean renameIndexFile( final FileSystem outputFS, final Path indexZipFilePath, final Path finalIndexZipFilePath diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePathTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePathTest.java new file mode 100644 index 000000000000..3dcd2033e2ff --- /dev/null +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePathTest.java @@ -0,0 +1,185 @@ +/* + * 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.druid.indexer; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class DataSegmentAndIndexZipFilePathTest +{ + private static final SegmentId SEGMENT_ID = SegmentId.dummy("data-source", 1); + private static final SegmentId OTHER_SEGMENT_ID = SegmentId.dummy("data-source2", 1); + private static final DataSegment SEGMENT = new DataSegment( + SEGMENT_ID, + null, + null, + null, + new NumberedShardSpec(1, 10), + null, + 0, + 0 + ); + private static final DataSegment OTHER_SEGMENT = new DataSegment( + OTHER_SEGMENT_ID, + null, + null, + null, + new NumberedShardSpec(1, 10), + null, + 0, + 0 + ); + + private DataSegmentAndIndexZipFilePath target; + + @Test + public void test_equals_otherNull_notEqual() + { + String tmpPath = "tmpPath"; + String finalPath = "finalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + Assert.assertNotEquals(target, null); + } + + @Test + public void test_equals_differentSegmentId_notEqual() + { + String tmpPath = "tmpPath"; + String finalPath = "finalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + + DataSegmentAndIndexZipFilePath other = new DataSegmentAndIndexZipFilePath( + OTHER_SEGMENT, + tmpPath, + finalPath + ); + Assert.assertNotEquals(target, other); + } + + @Test + public void test_equals_differentTmpPath_notEqual() + { + String tmpPath = "tmpPath"; + String otherTmpPath = "otherTmpPath"; + String finalPath = "finalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + + DataSegmentAndIndexZipFilePath other = new DataSegmentAndIndexZipFilePath( + SEGMENT, + otherTmpPath, + finalPath + ); + Assert.assertNotEquals(target, other); + } + + @Test + public void test_equals_differentFinalPath_notEqual() + { + String tmpPath = "tmpPath"; + String finalPath = "finalPath"; + String otherFinalPath = "otherFinalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + + DataSegmentAndIndexZipFilePath other = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + otherFinalPath + ); + Assert.assertNotEquals(target, other); + } + + @Test + public void test_equals_allFieldsEqualValue_equal() + { + String tmpPath = "tmpPath"; + String finalPath = "finalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + + DataSegmentAndIndexZipFilePath other = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + Assert.assertEquals(target, other); + } + + @Test + public void test_equals_sameObject_equal() + { + String tmpPath = "tmpPath"; + String finalPath = "finalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + + Assert.assertEquals(target, target); + } + + @Test + public void test_serde() throws IOException + { + String tmpPath = "tmpPath"; + String finalPath = "finalPath"; + target = new DataSegmentAndIndexZipFilePath( + SEGMENT, + tmpPath, + finalPath + ); + + final InjectableValues.Std injectableValues = new InjectableValues.Std(); + injectableValues.addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT); + final ObjectMapper mapper = new DefaultObjectMapper(); + mapper.setInjectableValues(injectableValues); + final String json = mapper.writeValueAsString(target); + final DataSegmentAndIndexZipFilePath fromJson = + mapper.readValue(json, DataSegmentAndIndexZipFilePath.class); + Assert.assertEquals(target, fromJson); + } +} diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index 3188762bc0b0..53cc606fdc71 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -61,6 +61,7 @@ import java.net.URISyntaxException; import java.nio.file.Files; import java.util.HashMap; +import java.util.List; import java.util.Map; /** @@ -299,6 +300,13 @@ public void progress() Assert.fail("Exception was not thrown for malicious zip file"); } + @Test + public void test_renameIndexFilesForSegments_emptySegments() throws IOException + { + List segmentAndIndexZipFilePaths = ImmutableList.of(); + JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); + } + private static class HadoopDruidIndexerConfigSpy extends HadoopDruidIndexerConfig { diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/MetadataStorageUpdaterJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/MetadataStorageUpdaterJobTest.java new file mode 100644 index 000000000000..0b867630cd10 --- /dev/null +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/MetadataStorageUpdaterJobTest.java @@ -0,0 +1,82 @@ +/* + * 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.druid.indexer; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.indexer.updater.MetadataStorageUpdaterJobSpec; +import org.easymock.EasyMock; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; +import java.util.stream.Collectors; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({ + IndexGeneratorJob.class +}) +@PowerMockIgnore({"javax.net.ssl.*"}) +public class MetadataStorageUpdaterJobTest +{ + private static final List DATA_SEGMENT_AND_INDEX_ZIP_FILE_PATHS = ImmutableList.of( + new DataSegmentAndIndexZipFilePath(null, null, null) + ); + private static final String SEGMENT_TABLE = "segments"; + private HadoopIngestionSpec spec; + private HadoopIOConfig ioConfig; + private MetadataStorageUpdaterJobSpec metadataUpdateSpec; + private HadoopDruidIndexerConfig config; + private MetadataStorageUpdaterJobHandler handler; + private MetadataStorageUpdaterJob target; + + @Test + public void test_run() + { + metadataUpdateSpec = PowerMock.createMock(MetadataStorageUpdaterJobSpec.class); + ioConfig = PowerMock.createMock(HadoopIOConfig.class); + spec = PowerMock.createMock(HadoopIngestionSpec.class); + config = PowerMock.createMock(HadoopDruidIndexerConfig.class); + handler = PowerMock.createMock(MetadataStorageUpdaterJobHandler.class); + PowerMock.mockStaticNice(IndexGeneratorJob.class); + + EasyMock.expect(metadataUpdateSpec.getSegmentTable()).andReturn(SEGMENT_TABLE); + EasyMock.expect(ioConfig.getMetadataUpdateSpec()).andReturn(metadataUpdateSpec); + EasyMock.expect(spec.getIOConfig()).andReturn(ioConfig); + EasyMock.expect(config.getSchema()).andReturn(spec); + EasyMock.expect(IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(config)) + .andReturn(DATA_SEGMENT_AND_INDEX_ZIP_FILE_PATHS); + handler.publishSegments( + SEGMENT_TABLE, + DATA_SEGMENT_AND_INDEX_ZIP_FILE_PATHS.stream().map(s -> s.getSegment()).collect( + Collectors.toList()), HadoopDruidIndexerConfig.JSON_MAPPER); + EasyMock.expectLastCall(); + target = new MetadataStorageUpdaterJob(config, handler); + + PowerMock.replayAll(); + + target.run(); + + PowerMock.verifyAll(); + } +} From 2d41899b81054679902bca98d1fcccd670ee12c8 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Thu, 8 Apr 2021 15:19:13 -0400 Subject: [PATCH 11/20] * add java docs --- .../druid/indexer/DataSegmentAndIndexZipFilePath.java | 6 ++++++ .../src/main/java/org/apache/druid/indexer/JobHelper.java | 7 ++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java index e6a7117ea365..e12f7fbf5dc6 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DataSegmentAndIndexZipFilePath.java @@ -23,8 +23,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.timeline.DataSegment; +import java.util.List; import java.util.Objects; +/** + * holds a {@link DataSegment} with the temporary file path where the corresponding index zip file is currently stored + * and the final path where the index zip file should eventually be moved to. + * see {@link JobHelper#renameIndexFilesForSegments(HadoopIngestionSpec, List)} + */ public class DataSegmentAndIndexZipFilePath { private final DataSegment segment; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index 67f8ce75cff6..d589df897039 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -629,7 +629,12 @@ public static Path makeTmpPath( /** * Renames the index files for the segments. This works around some limitations of both FileContext (no s3n support) and NativeS3FileSystem.rename - * which will not overwrite + * which will not overwrite. Note: segments should be renamed in the index task, not in a hadoop job, as race + * conditions between job retries can cause the final segment index file path to get clobbered. + * + * @param indexerSchema the hadoop ingestion spec + * @param segmentAndIndexZipFilePaths the list of segments with their currently stored tmp path and the final path + * that they should be renamed to. */ public static void renameIndexFilesForSegments( HadoopIngestionSpec indexerSchema, From 0b4bfbd1812d659dc1ac5241b1e2a0fc2f8e6047 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Fri, 9 Apr 2021 01:33:55 -0400 Subject: [PATCH 12/20] * add more unit tests --- .../druid/indexer/FileSystemHelper.java | 34 ++++ .../org/apache/druid/indexer/JobHelper.java | 2 +- .../indexer/HadoopDruidIndexerJobTest.java | 76 ++++++++ .../druid/indexer/JobHelperPowerMockTest.java | 181 ++++++++++++++++++ .../apache/druid/indexer/JobHelperTest.java | 8 - 5 files changed, 292 insertions(+), 9 deletions(-) create mode 100644 indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java create mode 100644 indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java create mode 100644 indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java new file mode 100644 index 000000000000..8834d2eefb7d --- /dev/null +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java @@ -0,0 +1,34 @@ +/* + * 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.druid.indexer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.io.IOException; +import java.net.URI; + +public class FileSystemHelper +{ + public static FileSystem get(URI uri, Configuration conf) throws IOException + { + return FileSystem.get(uri, conf); + } +} diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index d589df897039..a8ead3edc0cd 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -648,7 +648,7 @@ public static void renameIndexFilesForSegments( for (DataSegmentAndIndexZipFilePath segmentAndIndexZipFilePath : segmentAndIndexZipFilePaths) { Path tmpPath = new Path(segmentAndIndexZipFilePath.getTmpIndexZipFilePath()); Path finalIndexZipFilePath = new Path(segmentAndIndexZipFilePath.getFinalIndexZipFilePath()); - final FileSystem outputFS = FileSystem.get(finalIndexZipFilePath.toUri(), configuration); + final FileSystem outputFS = FileSystemHelper.get(finalIndexZipFilePath.toUri(), configuration); if (!renameIndexFile(outputFS, tmpPath, finalIndexZipFilePath)) { throw new IOE( "Unable to rename [%s] to [%s]", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java new file mode 100644 index 000000000000..3d11ab5e2249 --- /dev/null +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java @@ -0,0 +1,76 @@ +/* + * 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.druid.indexer; + +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.List; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({ + JobHelper.class, + IndexGeneratorJob.class +}) +@PowerMockIgnore({"javax.net.ssl.*"}) +public class HadoopDruidIndexerJobTest +{ + private HadoopDruidIndexerConfig config; + private MetadataStorageUpdaterJobHandler handler; + private HadoopDruidIndexerJob target; + + @Test + public void test_run() + { + config = PowerMock.createMock(HadoopDruidIndexerConfig.class); + handler = PowerMock.createMock(MetadataStorageUpdaterJobHandler.class); + PowerMock.mockStaticNice(JobHelper.class); + PowerMock.mockStaticNice(IndexGeneratorJob.class); + config.verify(); + EasyMock.expectLastCall(); + EasyMock.expect(config.isUpdaterJobSpecSet()).andReturn(false).anyTimes(); + config.setHadoopJobIdFileName(EasyMock.anyString()); + EasyMock.expectLastCall(); + JobHelper.ensurePaths(config); + EasyMock.expectLastCall(); + Capture> capturedJobs = Capture.newInstance(); + EasyMock.expect(JobHelper.runJobs(EasyMock.capture(capturedJobs), EasyMock.eq(config))).andReturn(true); + EasyMock.expect(IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(EasyMock.anyObject())).andReturn(null); + + + PowerMock.replayAll(); + + target = new HadoopDruidIndexerJob(config, handler); + target.run(); + + List jobs = capturedJobs.getValue(); + Assert.assertEquals(2, jobs.size()); + jobs.stream().filter(job -> !(job instanceof IndexGeneratorJob)).forEach(job -> Assert.assertTrue(job.run())); + + PowerMock.verifyAll(); + } +} diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java new file mode 100644 index 000000000000..71ab25588b02 --- /dev/null +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java @@ -0,0 +1,181 @@ +/* + * 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.druid.indexer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.path.StaticPathSpec; +import org.apache.druid.java.util.common.IOE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.easymock.EasyMock; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.io.IOException; +import java.net.URI; +import java.util.List; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({ + FileSystemHelper.class +}) +@PowerMockIgnore({"javax.net.ssl.*"}) +public class JobHelperPowerMockTest +{ + private static final DataSegment DATA_SEGMENT = new DataSegment( + "test1", + Intervals.of("2000/3000"), + "ver", + ImmutableMap.of( + "type", "google", + "bucket", "test-test", + "path", "tmp/foo:bar/index1.zip" + ), + ImmutableList.of(), + ImmutableList.of(), + NoneShardSpec.instance(), + 9, + 1024 + ); + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + private static final DataSchema DATA_SCHEMA = new DataSchema( + "test_ds", + JSON_MAPPER.convertValue( + new HadoopyStringInputRowParser( + new JSONParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + null + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + JSON_MAPPER + ); + + private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( + JSON_MAPPER.convertValue( + new StaticPathSpec("dummyPath", null), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), + null, + "dummyOutputPath" + ); + + private static final HadoopTuningConfig TUNING_CONFIG = HadoopTuningConfig + .makeDefaultTuningConfig() + .withWorkingPath("dummyWorkingPath"); + private static final HadoopIngestionSpec DUMMY_SPEC = new HadoopIngestionSpec(DATA_SCHEMA, IO_CONFIG, TUNING_CONFIG); + + @Test + public void test_renameIndexFilesForSegments_emptySegments() throws IOException + { + List segmentAndIndexZipFilePaths = ImmutableList.of(); + JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); + } + + @Test + public void test_renameIndexFilesForSegments_segmentIndexFileRenamedSuccessfully() + throws IOException + { + String tmpPath = "/tmp/index.zip.0"; + String finalPath = "/final/index.zip.0"; + List segmentAndIndexZipFilePaths = ImmutableList.of( + new DataSegmentAndIndexZipFilePath( + DATA_SEGMENT, + tmpPath, + finalPath + ) + ); + PowerMock.mockStaticNice(FileSystemHelper.class); + FileSystem fileSystem = PowerMock.createMock(FileSystem.class); + EasyMock.expect(FileSystemHelper.get( + EasyMock.anyObject(URI.class), + EasyMock.anyObject(Configuration.class) + )).andReturn(fileSystem); + EasyMock.expect(fileSystem.exists(EasyMock.anyObject(Path.class))).andReturn(false); + EasyMock.expect(fileSystem.rename(EasyMock.anyObject(Path.class), EasyMock.anyObject(Path.class))).andReturn(true); + + PowerMock.replayAll(); + + JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); + + PowerMock.verifyAll(); + } + + @Test (expected = IOE.class) + public void test_renameIndexFilesForSegments_segmentIndexFileRenamedFailed_throwsException() + throws IOException + { + String tmpPath = "/tmp/index.zip.0"; + String finalPath = "/final/index.zip.0"; + List segmentAndIndexZipFilePaths = ImmutableList.of( + new DataSegmentAndIndexZipFilePath( + DATA_SEGMENT, + tmpPath, + finalPath + ) + ); + PowerMock.mockStaticNice(FileSystemHelper.class); + FileSystem fileSystem = PowerMock.createMock(FileSystem.class); + EasyMock.expect(FileSystemHelper.get( + EasyMock.anyObject(URI.class), + EasyMock.anyObject(Configuration.class) + )).andReturn(fileSystem); + EasyMock.expect(fileSystem.exists(EasyMock.anyObject(Path.class))).andReturn(false); + EasyMock.expect(fileSystem.rename(EasyMock.anyObject(Path.class), EasyMock.anyObject(Path.class))).andReturn(false); + + PowerMock.replayAll(); + + JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); + + PowerMock.verifyAll(); + } +} diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index 53cc606fdc71..3188762bc0b0 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -61,7 +61,6 @@ import java.net.URISyntaxException; import java.nio.file.Files; import java.util.HashMap; -import java.util.List; import java.util.Map; /** @@ -300,13 +299,6 @@ public void progress() Assert.fail("Exception was not thrown for malicious zip file"); } - @Test - public void test_renameIndexFilesForSegments_emptySegments() throws IOException - { - List segmentAndIndexZipFilePaths = ImmutableList.of(); - JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); - } - private static class HadoopDruidIndexerConfigSpy extends HadoopDruidIndexerConfig { From c7fa3e83300ad446b9153ca752b88a18cfcf9217 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Fri, 9 Apr 2021 02:41:27 -0400 Subject: [PATCH 13/20] * Cleanup test --- .../druid/indexer/JobHelperPowerMockTest.java | 146 ++++++------------ 1 file changed, 49 insertions(+), 97 deletions(-) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java index 71ab25588b02..12b98f3bb853 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java @@ -19,25 +19,10 @@ package org.apache.druid.indexer; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.indexer.path.StaticPathSpec; import org.apache.druid.java.util.common.IOE; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.jackson.JacksonUtils; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -55,97 +40,46 @@ @RunWith(PowerMockRunner.class) @PrepareForTest({ - FileSystemHelper.class + FileSystemHelper.class, + HadoopDruidIndexerConfig.class }) @PowerMockIgnore({"javax.net.ssl.*"}) public class JobHelperPowerMockTest { - private static final DataSegment DATA_SEGMENT = new DataSegment( - "test1", - Intervals.of("2000/3000"), - "ver", - ImmutableMap.of( - "type", "google", - "bucket", "test-test", - "path", "tmp/foo:bar/index1.zip" - ), - ImmutableList.of(), - ImmutableList.of(), - NoneShardSpec.instance(), - 9, - 1024 - ); - private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - JSON_MAPPER.convertValue( - new HadoopyStringInputRowParser( - new JSONParseSpec( - new TimestampSpec("t", "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - null - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - JSON_MAPPER - ); - - private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( - JSON_MAPPER.convertValue( - new StaticPathSpec("dummyPath", null), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - null, - "dummyOutputPath" - ); - - private static final HadoopTuningConfig TUNING_CONFIG = HadoopTuningConfig - .makeDefaultTuningConfig() - .withWorkingPath("dummyWorkingPath"); - private static final HadoopIngestionSpec DUMMY_SPEC = new HadoopIngestionSpec(DATA_SCHEMA, IO_CONFIG, TUNING_CONFIG); + private static final String TMP_PATH = "/tmp/index.zip.0"; + private static final String FINAL_PATH = "/final/index.zip.0"; @Test public void test_renameIndexFilesForSegments_emptySegments() throws IOException { + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); List segmentAndIndexZipFilePaths = ImmutableList.of(); - JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); + + PowerMock.replayAll(); + + JobHelper.renameIndexFilesForSegments(ingestionSpec, segmentAndIndexZipFilePaths); + + PowerMock.verifyAll(); } @Test public void test_renameIndexFilesForSegments_segmentIndexFileRenamedSuccessfully() throws IOException { - String tmpPath = "/tmp/index.zip.0"; - String finalPath = "/final/index.zip.0"; + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); + mockFileSystem(true); + DataSegment segment = PowerMock.createMock(DataSegment.class); + List segmentAndIndexZipFilePaths = ImmutableList.of( new DataSegmentAndIndexZipFilePath( - DATA_SEGMENT, - tmpPath, - finalPath + segment, + TMP_PATH, + FINAL_PATH ) ); - PowerMock.mockStaticNice(FileSystemHelper.class); - FileSystem fileSystem = PowerMock.createMock(FileSystem.class); - EasyMock.expect(FileSystemHelper.get( - EasyMock.anyObject(URI.class), - EasyMock.anyObject(Configuration.class) - )).andReturn(fileSystem); - EasyMock.expect(fileSystem.exists(EasyMock.anyObject(Path.class))).andReturn(false); - EasyMock.expect(fileSystem.rename(EasyMock.anyObject(Path.class), EasyMock.anyObject(Path.class))).andReturn(true); - PowerMock.replayAll(); - JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); + JobHelper.renameIndexFilesForSegments(ingestionSpec, segmentAndIndexZipFilePaths); PowerMock.verifyAll(); } @@ -154,15 +88,38 @@ public void test_renameIndexFilesForSegments_segmentIndexFileRenamedSuccessfully public void test_renameIndexFilesForSegments_segmentIndexFileRenamedFailed_throwsException() throws IOException { - String tmpPath = "/tmp/index.zip.0"; - String finalPath = "/final/index.zip.0"; + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); + mockFileSystem(false); + DataSegment segment = PowerMock.createMock(DataSegment.class); List segmentAndIndexZipFilePaths = ImmutableList.of( new DataSegmentAndIndexZipFilePath( - DATA_SEGMENT, - tmpPath, - finalPath + segment, + TMP_PATH, + FINAL_PATH ) ); + + PowerMock.replayAll(); + + JobHelper.renameIndexFilesForSegments(ingestionSpec, segmentAndIndexZipFilePaths); + + PowerMock.verifyAll(); + } + + private HadoopIngestionSpec mockIngestionSpec() + { + HadoopDruidIndexerConfig indexerConfig = PowerMock.createMock(HadoopDruidIndexerConfig.class); + HadoopIngestionSpec ingestionSpec = PowerMock.createMock(HadoopIngestionSpec.class); + PowerMock.mockStaticNice(HadoopDruidIndexerConfig.class); + EasyMock.expect(indexerConfig.getAllowedProperties()).andReturn(ImmutableMap.of()).anyTimes(); + indexerConfig.addJobProperties(EasyMock.anyObject(Configuration.class)); + EasyMock.expectLastCall(); + EasyMock.expect(HadoopDruidIndexerConfig.fromSpec(ingestionSpec)).andReturn(indexerConfig); + return ingestionSpec; + } + + private void mockFileSystem(boolean renameSuccess) throws IOException + { PowerMock.mockStaticNice(FileSystemHelper.class); FileSystem fileSystem = PowerMock.createMock(FileSystem.class); EasyMock.expect(FileSystemHelper.get( @@ -170,12 +127,7 @@ public void test_renameIndexFilesForSegments_segmentIndexFileRenamedFailed_throw EasyMock.anyObject(Configuration.class) )).andReturn(fileSystem); EasyMock.expect(fileSystem.exists(EasyMock.anyObject(Path.class))).andReturn(false); - EasyMock.expect(fileSystem.rename(EasyMock.anyObject(Path.class), EasyMock.anyObject(Path.class))).andReturn(false); - - PowerMock.replayAll(); - - JobHelper.renameIndexFilesForSegments(DUMMY_SPEC, segmentAndIndexZipFilePaths); - - PowerMock.verifyAll(); + EasyMock.expect(fileSystem.rename(EasyMock.anyObject(Path.class), EasyMock.anyObject(Path.class))) + .andReturn(renameSuccess); } } From cf26d35a8f2a8804ec867fcef0a1979daf425336 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Fri, 9 Apr 2021 18:31:46 -0400 Subject: [PATCH 14/20] * Move removing of workingPath to index task rather than in hadoop job. --- .../HadoopDruidDetermineConfigurationJob.java | 2 +- .../druid/indexer/HadoopDruidIndexerJob.java | 2 +- .../org/apache/druid/indexer/JobHelper.java | 35 +- .../indexer/BatchDeltaIngestionTest.java | 10 +- .../indexer/HadoopDruidIndexerJobTest.java | 2 +- .../druid/indexer/IndexGeneratorJobTest.java | 10 +- .../druid/indexer/JobHelperPowerMockTest.java | 87 +++- .../indexing/common/task/HadoopIndexTask.java | 383 +++++++++++------- .../druid/cli/CliInternalHadoopIndexer.java | 2 +- 9 files changed, 364 insertions(+), 169 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java index 8b5b4b6b0bba..c046f6b2994b 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -59,7 +59,7 @@ public boolean run() if (config.isDeterminingPartitions()) { job = createPartitionJob(config); config.setHadoopJobIdFileName(hadoopJobIdFile); - return JobHelper.runSingleJob(job, config); + return JobHelper.runSingleJob(job); } else { final PartitionsSpec partitionsSpec = config.getPartitionsSpec(); final int shardsPerInterval; diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java index f5a22782e02d..58977ad48401 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidIndexerJob.java @@ -97,7 +97,7 @@ public boolean run() ); config.setHadoopJobIdFileName(hadoopJobIdFile); - return JobHelper.runJobs(jobs, config); + return JobHelper.runJobs(jobs); } @Override diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index a8ead3edc0cd..e8c17c93010d 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -386,29 +386,13 @@ public static void writeJobIdToFile(String hadoopJobIdFileName, String hadoopJob } } - public static boolean runSingleJob(Jobby job, HadoopDruidIndexerConfig config) + public static boolean runSingleJob(Jobby job) { boolean succeeded = job.run(); - - if (!config.getSchema().getTuningConfig().isLeaveIntermediate()) { - if (succeeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { - Path workingPath = config.makeIntermediatePath(); - log.info("Deleting path[%s]", workingPath); - try { - Configuration conf = injectSystemProperties(new Configuration(), config); - config.addJobProperties(conf); - workingPath.getFileSystem(conf).delete(workingPath, true); - } - catch (IOException e) { - log.error(e, "Failed to cleanup path[%s]", workingPath); - } - } - } - return succeeded; } - public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) + public static boolean runJobs(List jobs) { boolean succeeded = true; for (Jobby job : jobs) { @@ -418,8 +402,19 @@ public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) } } + return succeeded; + } + + public static void maybeDeleteIntermediatePath( + boolean indexerGeneratorJobSucceeded, + HadoopIngestionSpec indexerSchema) + { + HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec(indexerSchema); + final Configuration configuration = JobHelper.injectSystemProperties(new Configuration(), config); + config.addJobProperties(configuration); + JobHelper.injectDruidProperties(configuration, config); if (!config.getSchema().getTuningConfig().isLeaveIntermediate()) { - if (succeeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { + if (indexerGeneratorJobSucceeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { Path workingPath = config.makeIntermediatePath(); log.info("Deleting path[%s]", workingPath); try { @@ -432,8 +427,6 @@ public static boolean runJobs(List jobs, HadoopDruidIndexerConfig config) } } } - - return succeeded; } public static DataSegmentAndIndexZipFilePath serializeOutIndex( diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 644ae240262d..ad4ff6e9d903 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -372,7 +372,15 @@ private void testIngestion( ) throws Exception { IndexGeneratorJob job = new IndexGeneratorJob(config); - Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job), config)); + Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job))); + + List dataSegmentAndIndexZipFilePaths = + IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(config); + JobHelper.renameIndexFilesForSegments(config.getSchema(), dataSegmentAndIndexZipFilePaths); + + JobHelper.maybeDeleteIntermediatePath(true, config.getSchema()); + File workingPath = new File(config.makeIntermediatePath().toUri().getPath()); + Assert.assertFalse(workingPath.exists()); File segmentFolder = new File( StringUtils.format( diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java index 3d11ab5e2249..8231b7ce4aae 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerJobTest.java @@ -58,7 +58,7 @@ public void test_run() JobHelper.ensurePaths(config); EasyMock.expectLastCall(); Capture> capturedJobs = Capture.newInstance(); - EasyMock.expect(JobHelper.runJobs(EasyMock.capture(capturedJobs), EasyMock.eq(config))).andReturn(true); + EasyMock.expect(JobHelper.runJobs(EasyMock.capture(capturedJobs))).andReturn(true); EasyMock.expect(IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(EasyMock.anyObject())).andReturn(null); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index 3b4e6cb5d0f3..f5dea24d592d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -620,7 +620,7 @@ public void testIndexGeneratorJob() throws IOException private void verifyJob(IndexGeneratorJob job) throws IOException { - Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job), config)); + Assert.assertTrue(JobHelper.runJobs(ImmutableList.of(job))); final Map> intervalToSegments = new HashMap<>(); IndexGeneratorJob @@ -628,6 +628,14 @@ private void verifyJob(IndexGeneratorJob job) throws IOException .forEach(segmentAndIndexZipFilePath -> intervalToSegments.computeIfAbsent(segmentAndIndexZipFilePath.getSegment().getInterval(), k -> new ArrayList<>()) .add(segmentAndIndexZipFilePath.getSegment())); + List dataSegmentAndIndexZipFilePaths = + IndexGeneratorJob.getPublishedSegmentAndIndexZipFilePaths(config); + JobHelper.renameIndexFilesForSegments(config.getSchema(), dataSegmentAndIndexZipFilePaths); + + JobHelper.maybeDeleteIntermediatePath(true, config.getSchema()); + File workingPath = new File(config.makeIntermediatePath().toUri().getPath()); + Assert.assertTrue(workingPath.exists()); + final Map> intervalToIndexFiles = new HashMap<>(); int segmentNum = 0; for (DateTime currTime = interval.getStart(); currTime.isBefore(interval.getEnd()); currTime = currTime.plusDays(1)) { diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java index 12b98f3bb853..48f653a8c3b9 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperPowerMockTest.java @@ -49,6 +49,8 @@ public class JobHelperPowerMockTest private static final String TMP_PATH = "/tmp/index.zip.0"; private static final String FINAL_PATH = "/final/index.zip.0"; + private HadoopDruidIndexerConfig indexerConfig; + @Test public void test_renameIndexFilesForSegments_emptySegments() throws IOException { @@ -106,15 +108,96 @@ public void test_renameIndexFilesForSegments_segmentIndexFileRenamedFailed_throw PowerMock.verifyAll(); } + @Test + public void test_maybeDeleteIntermediatePath_leaveIntermediate_doesNotDeleteIntermediatePath() + { + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); + HadoopTuningConfig tuningConfig = PowerMock.createMock(HadoopTuningConfig.class); + EasyMock.expect(tuningConfig.isLeaveIntermediate()).andReturn(true); + EasyMock.expect(ingestionSpec.getTuningConfig()).andReturn(tuningConfig); + + PowerMock.replayAll(); + + JobHelper.maybeDeleteIntermediatePath(true, ingestionSpec); + + PowerMock.verifyAll(); + } + + @Test + public void test_maybeDeleteIntermediatePath_doNotleaveIntermediateAndIndexerJobSucceeded_deleteIntermediatePath() + throws IOException + { + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); + HadoopTuningConfig tuningConfig = PowerMock.createMock(HadoopTuningConfig.class); + Path workingPath = PowerMock.createMock(Path.class); + FileSystem workingPathFs = PowerMock.createMock(FileSystem.class); + EasyMock.expect(tuningConfig.isLeaveIntermediate()).andReturn(false); + EasyMock.expect(ingestionSpec.getTuningConfig()).andReturn(tuningConfig); + EasyMock.expect(workingPathFs.delete(workingPath, true)).andReturn(true); + EasyMock.expect(workingPath.getFileSystem(EasyMock.anyObject())).andReturn(workingPathFs); + EasyMock.expect(indexerConfig.makeIntermediatePath()).andReturn(workingPath); + + PowerMock.replayAll(); + + JobHelper.maybeDeleteIntermediatePath(true, ingestionSpec); + + PowerMock.verifyAll(); + } + + @Test + public void test_maybeDeleteIntermediatePath_doNotleaveIntermediateAndIndexJobFailedAndCleanupOnFailure_deleteIntermediatePath() + throws IOException + { + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); + HadoopTuningConfig tuningConfig = PowerMock.createMock(HadoopTuningConfig.class); + Path workingPath = PowerMock.createMock(Path.class); + FileSystem workingPathFs = PowerMock.createMock(FileSystem.class); + EasyMock.expect(tuningConfig.isLeaveIntermediate()).andReturn(false); + EasyMock.expect(tuningConfig.isCleanupOnFailure()).andReturn(true); + EasyMock.expect(ingestionSpec.getTuningConfig()).andReturn(tuningConfig).anyTimes(); + EasyMock.expect(workingPathFs.delete(workingPath, true)).andReturn(true); + EasyMock.expect(workingPath.getFileSystem(EasyMock.anyObject())).andReturn(workingPathFs); + EasyMock.expect(indexerConfig.makeIntermediatePath()).andReturn(workingPath); + + PowerMock.replayAll(); + + JobHelper.maybeDeleteIntermediatePath(false, ingestionSpec); + + PowerMock.verifyAll(); + } + + @Test + public void test_maybeDeleteIntermediatePath_deleteThrowsException_noExceptionPropogated() + throws IOException + { + HadoopIngestionSpec ingestionSpec = mockIngestionSpec(); + HadoopTuningConfig tuningConfig = PowerMock.createMock(HadoopTuningConfig.class); + Path workingPath = PowerMock.createMock(Path.class); + FileSystem workingPathFs = PowerMock.createMock(FileSystem.class); + EasyMock.expect(tuningConfig.isLeaveIntermediate()).andReturn(false); + EasyMock.expect(tuningConfig.isCleanupOnFailure()).andReturn(true); + EasyMock.expect(ingestionSpec.getTuningConfig()).andReturn(tuningConfig).anyTimes(); + EasyMock.expect(workingPathFs.delete(workingPath, true)).andThrow(new IOException("Delete Exception")); + EasyMock.expect(workingPath.getFileSystem(EasyMock.anyObject())).andReturn(workingPathFs); + EasyMock.expect(indexerConfig.makeIntermediatePath()).andReturn(workingPath); + + PowerMock.replayAll(); + + JobHelper.maybeDeleteIntermediatePath(false, ingestionSpec); + + PowerMock.verifyAll(); + } + private HadoopIngestionSpec mockIngestionSpec() { - HadoopDruidIndexerConfig indexerConfig = PowerMock.createMock(HadoopDruidIndexerConfig.class); + indexerConfig = PowerMock.createMock(HadoopDruidIndexerConfig.class); HadoopIngestionSpec ingestionSpec = PowerMock.createMock(HadoopIngestionSpec.class); PowerMock.mockStaticNice(HadoopDruidIndexerConfig.class); EasyMock.expect(indexerConfig.getAllowedProperties()).andReturn(ImmutableMap.of()).anyTimes(); indexerConfig.addJobProperties(EasyMock.anyObject(Configuration.class)); - EasyMock.expectLastCall(); + EasyMock.expectLastCall().anyTimes(); EasyMock.expect(HadoopDruidIndexerConfig.fromSpec(ingestionSpec)).andReturn(indexerConfig); + EasyMock.expect(indexerConfig.getSchema()).andReturn(ingestionSpec).anyTimes(); return ingestionSpec; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java index 8acf6756828d..962706059e4e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java @@ -31,6 +31,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import org.apache.commons.lang.BooleanUtils; import org.apache.druid.indexer.DataSegmentAndIndexZipFilePath; import org.apache.druid.indexer.HadoopDruidDetermineConfigurationJob; import org.apache.druid.indexer.HadoopDruidIndexerConfig; @@ -309,171 +310,184 @@ public TaskStatus runTask(TaskToolbox toolbox) @SuppressWarnings("unchecked") private TaskStatus runInternal(TaskToolbox toolbox) throws Exception { - registerResourceCloserOnAbnormalExit(config -> killHadoopJob()); - String hadoopJobIdFile = getHadoopJobIdFileName(); - final ClassLoader loader = buildClassLoader(toolbox); - boolean determineIntervals = spec.getDataSchema().getGranularitySpec().inputIntervals().isEmpty(); - - HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed( - spec, - jsonMapper, - new OverlordActionBasedUsedSegmentsRetriever(toolbox) - ); - - Object determinePartitionsInnerProcessingRunner = getForeignClassloaderObject( - "org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopDetermineConfigInnerProcessingRunner", - loader - ); - determinePartitionsStatsGetter = new InnerProcessingStatsGetter(determinePartitionsInnerProcessingRunner); - - String[] determinePartitionsInput = new String[]{ - toolbox.getJsonMapper().writeValueAsString(spec), - toolbox.getConfig().getHadoopWorkingPath(), - toolbox.getSegmentPusher().getPathForHadoop(), - hadoopJobIdFile - }; - - HadoopIngestionSpec indexerSchema; - final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); - Class determinePartitionsRunnerClass = determinePartitionsInnerProcessingRunner.getClass(); - Method determinePartitionsInnerProcessingRunTask = determinePartitionsRunnerClass.getMethod( - "runTask", - determinePartitionsInput.getClass() - ); + boolean indexGeneratorJobAttempted = false; + boolean indexGeneratorJobSuccess = false; + HadoopIngestionSpec indexerSchema = null; try { - Thread.currentThread().setContextClassLoader(loader); + registerResourceCloserOnAbnormalExit(config -> killHadoopJob()); + String hadoopJobIdFile = getHadoopJobIdFileName(); + final ClassLoader loader = buildClassLoader(toolbox); + boolean determineIntervals = spec.getDataSchema().getGranularitySpec().inputIntervals().isEmpty(); - ingestionState = IngestionState.DETERMINE_PARTITIONS; + HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed( + spec, + jsonMapper, + new OverlordActionBasedUsedSegmentsRetriever(toolbox) + ); - final String determineConfigStatusString = (String) determinePartitionsInnerProcessingRunTask.invoke( - determinePartitionsInnerProcessingRunner, - new Object[]{determinePartitionsInput} + Object determinePartitionsInnerProcessingRunner = getForeignClassloaderObject( + "org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopDetermineConfigInnerProcessingRunner", + loader ); + determinePartitionsStatsGetter = new InnerProcessingStatsGetter(determinePartitionsInnerProcessingRunner); + String[] determinePartitionsInput = new String[]{ + toolbox.getJsonMapper().writeValueAsString(spec), + toolbox.getConfig().getHadoopWorkingPath(), + toolbox.getSegmentPusher().getPathForHadoop(), + hadoopJobIdFile + }; - determineConfigStatus = toolbox - .getJsonMapper() - .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); + final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); + Class determinePartitionsRunnerClass = determinePartitionsInnerProcessingRunner.getClass(); + Method determinePartitionsInnerProcessingRunTask = determinePartitionsRunnerClass.getMethod( + "runTask", + determinePartitionsInput.getClass() + ); + try { + Thread.currentThread().setContextClassLoader(loader); - indexerSchema = determineConfigStatus.getSchema(); - if (indexerSchema == null) { - errorMsg = determineConfigStatus.getErrorMsg(); - toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); - return TaskStatus.failure( - getId(), - errorMsg + ingestionState = IngestionState.DETERMINE_PARTITIONS; + + final String determineConfigStatusString = (String) determinePartitionsInnerProcessingRunTask.invoke( + determinePartitionsInnerProcessingRunner, + new Object[]{determinePartitionsInput} ); - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - Thread.currentThread().setContextClassLoader(oldLoader); - } - // We should have a lock from before we started running only if interval was specified - String version; - if (determineIntervals) { - Interval interval = JodaUtils.umbrellaInterval( - JodaUtils.condenseIntervals( - indexerSchema.getDataSchema().getGranularitySpec().sortedBucketIntervals() - ) - ); - final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT_MILLIS); - // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, the below line can incur http timeout error. - final TaskLock lock = Preconditions.checkNotNull( - toolbox.getTaskActionClient().submit( - new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs) - ), - "Cannot acquire a lock for interval[%s]", interval - ); - version = lock.getVersion(); - } else { - Iterable locks = getTaskLocks(toolbox.getTaskActionClient()); - final TaskLock myLock = Iterables.getOnlyElement(locks); - version = myLock.getVersion(); - } - final String specVersion = indexerSchema.getTuningConfig().getVersion(); - if (indexerSchema.getTuningConfig().isUseExplicitVersion()) { - if (specVersion.compareTo(version) < 0) { - version = specVersion; - } else { - log.error( - "Spec version can not be greater than or equal to the lock version, Spec version: [%s] Lock version: [%s].", - specVersion, - version + determineConfigStatus = toolbox + .getJsonMapper() + .readValue(determineConfigStatusString, HadoopDetermineConfigInnerProcessingStatus.class); + + indexerSchema = determineConfigStatus.getSchema(); + if (indexerSchema == null) { + errorMsg = determineConfigStatus.getErrorMsg(); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg + ); + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } + + // We should have a lock from before we started running only if interval was specified + String version; + if (determineIntervals) { + Interval interval = JodaUtils.umbrellaInterval( + JodaUtils.condenseIntervals( + indexerSchema.getDataSchema().getGranularitySpec().sortedBucketIntervals() + ) ); - toolbox.getTaskReportFileWriter().write(getId(), null); - return TaskStatus.failure(getId()); + final long lockTimeoutMs = getContextValue(Tasks.LOCK_TIMEOUT_KEY, Tasks.DEFAULT_LOCK_TIMEOUT_MILLIS); + // Note: if lockTimeoutMs is larger than ServerConfig.maxIdleTime, the below line can incur http timeout error. + final TaskLock lock = Preconditions.checkNotNull( + toolbox.getTaskActionClient().submit( + new TimeChunkLockAcquireAction(TaskLockType.EXCLUSIVE, interval, lockTimeoutMs) + ), + "Cannot acquire a lock for interval[%s]", interval + ); + version = lock.getVersion(); + } else { + Iterable locks = getTaskLocks(toolbox.getTaskActionClient()); + final TaskLock myLock = Iterables.getOnlyElement(locks); + version = myLock.getVersion(); } - } - log.info("Setting version to: %s", version); + final String specVersion = indexerSchema.getTuningConfig().getVersion(); + if (indexerSchema.getTuningConfig().isUseExplicitVersion()) { + if (specVersion.compareTo(version) < 0) { + version = specVersion; + } else { + log.error( + "Spec version can not be greater than or equal to the lock version, Spec version: [%s] Lock version: [%s].", + specVersion, + version + ); + toolbox.getTaskReportFileWriter().write(getId(), null); + return TaskStatus.failure(getId()); + } + } - Object innerProcessingRunner = getForeignClassloaderObject( - "org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopIndexGeneratorInnerProcessingRunner", - loader - ); - buildSegmentsStatsGetter = new InnerProcessingStatsGetter(innerProcessingRunner); + log.info("Setting version to: %s", version); + + Object innerProcessingRunner = getForeignClassloaderObject( + "org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopIndexGeneratorInnerProcessingRunner", + loader + ); + buildSegmentsStatsGetter = new InnerProcessingStatsGetter(innerProcessingRunner); - String[] buildSegmentsInput = new String[]{ - toolbox.getJsonMapper().writeValueAsString(indexerSchema), - version, - hadoopJobIdFile - }; + String[] buildSegmentsInput = new String[]{ + toolbox.getJsonMapper().writeValueAsString(indexerSchema), + version, + hadoopJobIdFile + }; - Class buildSegmentsRunnerClass = innerProcessingRunner.getClass(); - Method innerProcessingRunTask = buildSegmentsRunnerClass.getMethod("runTask", buildSegmentsInput.getClass()); + Class buildSegmentsRunnerClass = innerProcessingRunner.getClass(); + Method innerProcessingRunTask = buildSegmentsRunnerClass.getMethod("runTask", buildSegmentsInput.getClass()); - try { - Thread.currentThread().setContextClassLoader(loader); + try { + Thread.currentThread().setContextClassLoader(loader); - ingestionState = IngestionState.BUILD_SEGMENTS; - final String jobStatusString = (String) innerProcessingRunTask.invoke( - innerProcessingRunner, - new Object[]{buildSegmentsInput} - ); + ingestionState = IngestionState.BUILD_SEGMENTS; + indexGeneratorJobAttempted = true; + final String jobStatusString = (String) innerProcessingRunTask.invoke( + innerProcessingRunner, + new Object[]{buildSegmentsInput} + ); - buildSegmentsStatus = toolbox.getJsonMapper().readValue( - jobStatusString, - HadoopIndexGeneratorInnerProcessingStatus.class - ); + buildSegmentsStatus = toolbox.getJsonMapper().readValue( + jobStatusString, + HadoopIndexGeneratorInnerProcessingStatus.class + ); - List dataSegmentAndIndexZipFilePaths = buildSegmentsStatus.getDataSegmentAndIndexZipFilePaths(); - if (dataSegmentAndIndexZipFilePaths != null) { - try { - Thread.currentThread().setContextClassLoader(oldLoader); - renameSegmentIndexFilesJob( - toolbox.getJsonMapper().writeValueAsString(indexerSchema), - toolbox.getJsonMapper().writeValueAsString(dataSegmentAndIndexZipFilePaths) + List dataSegmentAndIndexZipFilePaths = buildSegmentsStatus.getDataSegmentAndIndexZipFilePaths(); + if (dataSegmentAndIndexZipFilePaths != null) { + indexGeneratorJobSuccess = true; + try { + Thread.currentThread().setContextClassLoader(oldLoader); + renameSegmentIndexFilesJob( + toolbox.getJsonMapper().writeValueAsString(indexerSchema), + toolbox.getJsonMapper().writeValueAsString(dataSegmentAndIndexZipFilePaths) + ); + } + finally { + Thread.currentThread().setContextClassLoader(loader); + } + + ingestionState = IngestionState.COMPLETED; + toolbox.publishSegments(dataSegmentAndIndexZipFilePaths.stream() + .map(DataSegmentAndIndexZipFilePath::getSegment) + .collect(Collectors.toList())); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); + return TaskStatus.success(getId()); + } else { + errorMsg = buildSegmentsStatus.getErrorMsg(); + toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); + return TaskStatus.failure( + getId(), + errorMsg ); } - finally { - Thread.currentThread().setContextClassLoader(loader); - } - - ingestionState = IngestionState.COMPLETED; - toolbox.publishSegments(dataSegmentAndIndexZipFilePaths.stream() - .map(DataSegmentAndIndexZipFilePath::getSegment) - .collect(Collectors.toList())); - toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); - return TaskStatus.success(getId()); - } else { - errorMsg = buildSegmentsStatus.getErrorMsg(); - toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports()); - return TaskStatus.failure( - getId(), - errorMsg - ); } - } - catch (Exception e) { - throw new RuntimeException(e); + catch (Exception e) { + throw new RuntimeException(e); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } } finally { - Thread.currentThread().setContextClassLoader(oldLoader); + indexerGeneratorCleanupJob( + indexGeneratorJobAttempted, + indexGeneratorJobSuccess, + indexerSchema == null ? null : toolbox.getJsonMapper().writeValueAsString(indexerSchema) + ); } } @@ -559,6 +573,54 @@ private void renameSegmentIndexFilesJob( } } + private void indexerGeneratorCleanupJob( + boolean indexGeneratorJobAttempted, + boolean indexGeneratorJobSuccess, + String hadoopIngestionSpecStr + ) + { + if (!indexGeneratorJobAttempted) { + log.info("No need for cleanup as index generator job did not even run"); + return; + } + + final ClassLoader oldLoader = Thread.currentThread().getContextClassLoader(); + try { + ClassLoader loader = HadoopTask.buildClassLoader( + getHadoopDependencyCoordinates(), + taskConfig.getDefaultHadoopCoordinates() + ); + + Object indexerGeneratorCleanupRunner = getForeignClassloaderObject( + "org.apache.druid.indexing.common.task.HadoopIndexTask$HadoopIndexerGeneratorCleanupRunner", + loader + ); + + String[] indexerGeneratorCleanupJobInput = new String[]{ + indexGeneratorJobSuccess ? "true" : "false", + hadoopIngestionSpecStr, + }; + + Class buildIndexerGeneratorCleanupRunnerClass = indexerGeneratorCleanupRunner.getClass(); + Method indexerGeneratorCleanup = buildIndexerGeneratorCleanupRunnerClass.getMethod( + "runTask", + indexerGeneratorCleanupJobInput.getClass() + ); + + Thread.currentThread().setContextClassLoader(loader); + indexerGeneratorCleanup.invoke( + indexerGeneratorCleanupRunner, + new Object[]{indexerGeneratorCleanupJobInput} + ); + } + catch (Exception e) { + log.warn(e, "Failed to cleanup after index generator job"); + } + finally { + Thread.currentThread().setContextClassLoader(oldLoader); + } + } + @GET @Path("/rowStats") @Produces(MediaType.APPLICATION_JSON) @@ -876,6 +938,47 @@ public void runTask(String[] args) throws Exception } } + @SuppressWarnings("unused") + public static class HadoopIndexerGeneratorCleanupRunner + { + TypeReference> LIST_DATA_SEGMENT_AND_INDEX_ZIP_FILE_PATH = + new TypeReference>() + { + }; + + public void runTask(String[] args) throws Exception + { + if (args.length != 2) { + log.warn("HadoopIndexerGeneratorCleanupRunner called with improper number of arguments"); + } + + String indexGeneratorJobSucceededStr = args[0]; + String hadoopIngestionSpecStr = args[1]; + + HadoopIngestionSpec indexerSchema; + boolean indexGeneratorJobSucceeded; + List dataSegmentAndIndexZipFilePaths; + try { + indexerSchema = HadoopDruidIndexerConfig.JSON_MAPPER.readValue( + hadoopIngestionSpecStr, + HadoopIngestionSpec.class + ); + indexGeneratorJobSucceeded = BooleanUtils.toBoolean(indexGeneratorJobSucceededStr); + } + catch (Exception e) { + log.warn( + e, + "HadoopIndexerGeneratorCleanupRunner: Error occurred while trying to read input parameters into data objects" + ); + throw e; + } + JobHelper.maybeDeleteIntermediatePath( + indexGeneratorJobSucceeded, + indexerSchema + ); + } + } + public static class HadoopIndexGeneratorInnerProcessingStatus { private final List dataSegmentAndIndexZipFilePaths; diff --git a/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java b/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java index 4235abbf884c..8129e991615b 100644 --- a/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java @@ -119,7 +119,7 @@ public void run() List jobs = new ArrayList<>(); jobs.add(new HadoopDruidDetermineConfigurationJob(config)); jobs.add(new HadoopDruidIndexerJob(config, injector.getInstance(MetadataStorageUpdaterJobHandler.class))); - JobHelper.runJobs(jobs, config); + JobHelper.runJobs(jobs); } catch (Exception e) { From ed2e4ff6163ba88e4a1c7aa01adcc6636f75f64f Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Mon, 12 Apr 2021 12:15:33 -0400 Subject: [PATCH 15/20] * Address review comments --- .../indexer/HadoopDruidDetermineConfigurationJob.java | 8 +++++++- .../org/apache/druid/cli/CliInternalHadoopIndexer.java | 7 +++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java index c046f6b2994b..9128c3182a0e 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; +import org.apache.hadoop.mapreduce.Job; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -59,7 +60,12 @@ public boolean run() if (config.isDeterminingPartitions()) { job = createPartitionJob(config); config.setHadoopJobIdFileName(hadoopJobIdFile); - return JobHelper.runSingleJob(job); + boolean jobSucceeded = JobHelper.runSingleJob(job); + JobHelper.maybeDeleteIntermediatePath( + jobSucceeded, + config.getSchema() + ); + return jobSucceeded; } else { final PartitionsSpec partitionsSpec = config.getPartitionsSpec(); final int shardsPerInterval; diff --git a/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java b/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java index 8129e991615b..b227ababb449 100644 --- a/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java @@ -117,9 +117,12 @@ public void run() ); List jobs = new ArrayList<>(); + HadoopDruidIndexerJob indexerJob = new HadoopDruidIndexerJob(config, injector.getInstance(MetadataStorageUpdaterJobHandler.class)); jobs.add(new HadoopDruidDetermineConfigurationJob(config)); - jobs.add(new HadoopDruidIndexerJob(config, injector.getInstance(MetadataStorageUpdaterJobHandler.class))); - JobHelper.runJobs(jobs); + jobs.add(indexerJob); + boolean jobsSucceeded = JobHelper.runJobs(jobs); + JobHelper.renameIndexFilesForSegments(config.getSchema(), indexerJob.getPublishedSegmentAndIndexZipFilePaths()); + JobHelper.maybeDeleteIntermediatePath(jobsSucceeded, config.getSchema()); } catch (Exception e) { From 3692fe6e13f9a760638cb688046fb6098bf1d72b Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Mon, 12 Apr 2021 13:46:59 -0400 Subject: [PATCH 16/20] * remove unused import --- .../druid/indexer/HadoopDruidDetermineConfigurationJob.java | 1 - 1 file changed, 1 deletion(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java index 9128c3182a0e..ea37db1a10ec 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopDruidDetermineConfigurationJob.java @@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; -import org.apache.hadoop.mapreduce.Job; import org.joda.time.DateTime; import org.joda.time.Interval; From 219ceb92f6073c8178bb5b25d0955cf3592320e4 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Tue, 13 Apr 2021 01:29:45 -0400 Subject: [PATCH 17/20] * Address review comments --- .../main/java/org/apache/druid/indexer/JobHelper.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index e8c17c93010d..e21fa5f147f1 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -406,7 +406,7 @@ public static boolean runJobs(List jobs) } public static void maybeDeleteIntermediatePath( - boolean indexerGeneratorJobSucceeded, + boolean jobSucceeded, HadoopIngestionSpec indexerSchema) { HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromSpec(indexerSchema); @@ -414,13 +414,12 @@ public static void maybeDeleteIntermediatePath( config.addJobProperties(configuration); JobHelper.injectDruidProperties(configuration, config); if (!config.getSchema().getTuningConfig().isLeaveIntermediate()) { - if (indexerGeneratorJobSucceeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { + if (jobSucceeded || config.getSchema().getTuningConfig().isCleanupOnFailure()) { Path workingPath = config.makeIntermediatePath(); log.info("Deleting path[%s]", workingPath); try { - Configuration conf = injectSystemProperties(new Configuration(), config); - config.addJobProperties(conf); - workingPath.getFileSystem(conf).delete(workingPath, true); + config.addJobProperties(configuration); + workingPath.getFileSystem(configuration).delete(workingPath, true); } catch (IOException e) { log.error(e, "Failed to cleanup path[%s]", workingPath); From 8d0ef203400547eeade0377e83491a43502f0e8d Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Fri, 16 Apr 2021 18:32:10 -0400 Subject: [PATCH 18/20] Do not overwrite segment descriptor for segment if it already exists. --- .../main/java/org/apache/druid/indexer/JobHelper.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java index e21fa5f147f1..b8c29b82f7a7 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/JobHelper.java @@ -499,9 +499,12 @@ public long push() throws IOException try { progressable.progress(); if (outputFS.exists(descriptorPath)) { - if (!outputFS.delete(descriptorPath, false)) { - throw new IOE("Failed to delete descriptor at [%s]", descriptorPath); - } + // If the descriptor path already exists, don't overwrite, and risk clobbering it. + // If it already exists, it means that the segment data is already written to the + // tmp path, and the existing descriptor written should give us the information we + // need to rename the segment index to final path and publish it in the top level task. + log.info("descriptor path [%s] already exists, not overwriting", descriptorPath); + return -1; } try (final OutputStream descriptorOut = outputFS.create( descriptorPath, From 883402a53fbd7a8d6679ebc06379a6a2f5a31686 Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Tue, 20 Apr 2021 09:31:58 -1000 Subject: [PATCH 19/20] * add comments to FileSystemHelper class --- .../main/java/org/apache/druid/indexer/FileSystemHelper.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java index 8834d2eefb7d..96fde6b8ecee 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/FileSystemHelper.java @@ -25,6 +25,10 @@ import java.io.IOException; import java.net.URI; +/** + * This class exists for testing purposes, see {@link JobHelperPowerMockTest}. Using the + * raw {@link FileSystem} class resulted in errors with java assist. + */ public class FileSystemHelper { public static FileSystem get(URI uri, Configuration conf) throws IOException From de4827636bc809f64651b0b4c88198791b2ec8ea Mon Sep 17 00:00:00 2001 From: zachary sherman Date: Tue, 20 Apr 2021 18:30:26 -1000 Subject: [PATCH 20/20] * fix local hadoop integration test --- integration-tests/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 2fb9adc239e9..8ad9cec8ffac 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -102,6 +102,10 @@ javax.servlet servlet-api + + com.squareup.okhttp + okhttp +