private static void setJobName(JobConf jobConf, List<DataSegment> segments) { if (segments.size() == 1) { final DataSegment segment = segments.get(0); jobConf.setJobName( String.format( "druid-convert-%s-%s-%s", segment.getDataSource(), segment.getInterval(), segment.getVersion())); } else { final Set<String> dataSources = Sets.newHashSet( Iterables.transform( segments, new Function<DataSegment, String>() { @Override public String apply(DataSegment input) { return input.getDataSource(); } })); final Set<String> versions = Sets.newHashSet( Iterables.transform( segments, new Function<DataSegment, String>() { @Override public String apply(DataSegment input) { return input.getVersion(); } })); jobConf.setJobName( String.format( "druid-convert-%s-%s", Arrays.toString(dataSources.toArray()), Arrays.toString(versions.toArray()))); } }
@Override public Set<DataSegment> findUsedSegments(Set<SegmentIdentifier> identifiers) throws IOException { final VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(Ordering.natural()); for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { timeline.add( dataSegment.getInterval(), dataSegment.getVersion(), dataSegment.getShardSpec().createChunk(dataSegment)); } final Set<DataSegment> retVal = Sets.newHashSet(); for (SegmentIdentifier identifier : identifiers) { for (TimelineObjectHolder<String, DataSegment> holder : timeline.lookup(identifier.getInterval())) { for (PartitionChunk<DataSegment> chunk : holder.getObject()) { if (identifiers.contains(SegmentIdentifier.fromDataSegment(chunk.getObject()))) { retVal.add(chunk.getObject()); } } } } return retVal; }
@Override public void addSegment(DataSegment segment) { try { log.info("Loading segment %s", segment.getIdentifier()); try { serverManager.loadSegment(segment); } catch (Exception e) { removeSegment(segment); throw new SegmentLoadingException( e, "Exception loading segment[%s]", segment.getIdentifier()); } File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.exists()) { try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } catch (IOException e) { removeSegment(segment); throw new SegmentLoadingException( e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile); } } try { announcer.announceSegment(segment); } catch (IOException e) { throw new SegmentLoadingException( e, "Failed to announce segment[%s]", segment.getIdentifier()); } } catch (SegmentLoadingException e) { log.makeAlert(e, "Failed to load segment for dataSource").addData("segment", segment).emit(); } }
private void loadCache() { File baseDir = config.getInfoDir(); if (!baseDir.exists()) { return; } List<DataSegment> cachedSegments = Lists.newArrayList(); for (File file : baseDir.listFiles()) { log.info("Loading segment cache file [%s]", file); try { DataSegment segment = jsonMapper.readValue(file, DataSegment.class); if (serverManager.isSegmentCached(segment)) { cachedSegments.add(segment); } else { log.warn( "Unable to find cache file for %s. Deleting lookup entry", segment.getIdentifier()); File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } } } catch (Exception e) { log.makeAlert(e, "Failed to load segment from segmentInfo file") .addData("file", file) .emit(); } } addSegments(cachedSegments); }
@Test public void testV1SerializationNullMetrics() throws Exception { final DataSegment segment = DataSegment.builder() .dataSource("foo") .interval(new Interval("2012-01-01/2012-01-02")) .version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .build(); final DataSegment segment2 = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class); Assert.assertEquals("empty dimensions", ImmutableList.of(), segment2.getDimensions()); Assert.assertEquals("empty metrics", ImmutableList.of(), segment2.getMetrics()); }
@Test public void testIdentifierWithNonzeroPartition() { final DataSegment segment = DataSegment.builder() .dataSource("foo") .interval(new Interval("2012-01-01/2012-01-02")) .version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .shardSpec(new SingleDimensionShardSpec("bar", "abc", "def", 1)) .build(); Assert.assertEquals( "foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z_1", segment.getIdentifier()); }
@Test public void testIdentifier() { final DataSegment segment = DataSegment.builder() .dataSource("foo") .interval(new Interval("2012-01-01/2012-01-02")) .version(new DateTime("2012-01-01T11:22:33.444Z").toString()) .shardSpec(NoneShardSpec.instance()) .build(); Assert.assertEquals( "foo_2012-01-01T00:00:00.000Z_2012-01-02T00:00:00.000Z_2012-01-01T11:22:33.444Z", segment.getIdentifier()); }
@Override public void kill(DataSegment segment) throws SegmentLoadingException { final File path = getPath(segment); log.info("killing segment[%s] mapped to path[%s]", segment.getIdentifier(), path); try { if (path.getName().endsWith(".zip")) { // path format -- > .../dataSource/interval/version/partitionNum/xxx.zip File partitionNumDir = path.getParentFile(); FileUtils.deleteDirectory(partitionNumDir); // try to delete other directories if possible File versionDir = partitionNumDir.getParentFile(); if (versionDir.delete()) { File intervalDir = versionDir.getParentFile(); if (intervalDir.delete()) { File dataSourceDir = intervalDir.getParentFile(); dataSourceDir.delete(); } } } else { throw new SegmentLoadingException("Unknown file type[%s]", path); } } catch (IOException e) { throw new SegmentLoadingException(e, "Unable to kill segment"); } }
@Test public void testVersionConverterSubTaskSerde() throws Exception { final ConvertSegmentTask.SubTask task = new ConvertSegmentTask.SubTask( "myGroupId", DataSegment.builder() .dataSource("foo") .interval(new Interval("2010-01-01/P1D")) .version("1234") .build(), indexSpec, false, true, null); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change final ConvertSegmentTask.SubTask task2 = (ConvertSegmentTask.SubTask) jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals("myGroupId", task.getGroupId()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getSegment(), task2.getSegment()); }
@Test public void testBucketMonthComparator() throws Exception { DataSegment[] sortedOrder = { makeDataSegment("test1", "2011-01-01/2011-01-02", "a"), makeDataSegment("test1", "2011-01-02/2011-01-03", "a"), makeDataSegment("test1", "2011-01-02/2011-01-03", "b"), makeDataSegment("test2", "2011-01-01/2011-01-02", "a"), makeDataSegment("test2", "2011-01-02/2011-01-03", "a"), makeDataSegment("test1", "2011-02-01/2011-02-02", "a"), makeDataSegment("test1", "2011-02-02/2011-02-03", "a"), makeDataSegment("test1", "2011-02-02/2011-02-03", "b"), makeDataSegment("test2", "2011-02-01/2011-02-02", "a"), makeDataSegment("test2", "2011-02-02/2011-02-03", "a"), }; List<DataSegment> shuffled = Lists.newArrayList(sortedOrder); Collections.shuffle(shuffled); Set<DataSegment> theSet = Sets.newTreeSet(DataSegment.bucketMonthComparator()); theSet.addAll(shuffled); int index = 0; for (DataSegment dataSegment : theSet) { Assert.assertEquals(sortedOrder[index], dataSegment); ++index; } }
public void removeSegments(Iterable<DataSegment> segments) { try { for (DataSegment segment : segments) { serverManager.dropSegment(segment); File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.delete()) { log.warn("Unable to delete segmentInfoCacheFile[%s]", segmentInfoCacheFile); } } announcer.unannounceSegments(segments); } catch (Exception e) { log.makeAlert(e, "Failed to remove segments").addData("segments", segments).emit(); } }
@Test public void testAppendTaskSerde() throws Exception { final List<DataSegment> segments = ImmutableList.of( DataSegment.builder() .dataSource("foo") .interval(new Interval("2010-01-01/P1D")) .version("1234") .build(), DataSegment.builder() .dataSource("foo") .interval(new Interval("2010-01-02/P1D")) .version("5678") .build()); final AppendTask task = new AppendTask( null, "foo", segments, ImmutableList.<AggregatorFactory>of(new CountAggregatorFactory("cnt")), indexSpec, null); final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change final AppendTask task2 = (AppendTask) jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals(new Interval("2010-01-01/P2D"), task.getInterval()); Assert.assertEquals(task.getId(), task2.getId()); Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getInterval(), task2.getInterval()); Assert.assertEquals(task.getSegments(), task2.getSegments()); final AppendTask task3 = (AppendTask) jsonMapper.readValue( jsonMapper.writeValueAsString(new ClientAppendQuery("foo", segments)), Task.class); Assert.assertEquals("foo", task3.getDataSource()); Assert.assertEquals(new Interval("2010-01-01/P2D"), task3.getInterval()); Assert.assertEquals(task3.getSegments(), segments); Assert.assertEquals(task.getAggregators(), task2.getAggregators()); }
private DataSegment makeDataSegment(String dataSource, String interval, String version) { return DataSegment.builder() .dataSource(dataSource) .interval(new Interval(interval)) .version(version) .size(1) .build(); }
@Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { DatasourceWhitelist whitelist = whitelistRef.get(); for (DataSegment dataSegment : params.getAvailableSegments()) { if (whitelist == null || whitelist.contains(dataSegment.getDataSource())) { final Integer binaryVersion = dataSegment.getBinaryVersion(); if (binaryVersion == null || binaryVersion < IndexIO.CURRENT_VERSION_ID) { log.info("Upgrading version on segment[%s]", dataSegment.getIdentifier()); indexingServiceClient.upgradeSegment(dataSegment); } } } return params; }
public S3Coords(DataSegment segment) { Map<String, Object> loadSpec = segment.getLoadSpec(); bucket = MapUtils.getString(loadSpec, BUCKET); path = MapUtils.getString(loadSpec, KEY); if (path.startsWith("/")) { path = path.substring(1); } }
private static void corrupt(DataSegment segment) throws IOException { final Map<String, Object> localLoadSpec = segment.getLoadSpec(); final Path segmentPath = Paths.get(localLoadSpec.get("path").toString()); final MappedByteBuffer buffer = Files.map(segmentPath.toFile(), FileChannel.MapMode.READ_WRITE); while (buffer.hasRemaining()) { buffer.put((byte) 0xFF); } }
@Override public Set<DataSegment> perform(Task task, TaskActionToolbox toolbox) throws IOException { toolbox.verifyTaskLocksAndSinglePartitionSettitude(task, segments, true); final Set<DataSegment> retVal = toolbox.getIndexerDBCoordinator().announceHistoricalSegments(segments); // Emit metrics final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder().setUser2(task.getDataSource()).setUser4(task.getType()); for (DataSegment segment : segments) { metricBuilder.setUser5(segment.getInterval().toString()); toolbox.getEmitter().emit(metricBuilder.build("indexer/segment/bytes", segment.getSize())); } return retVal; }
@Test public void testSerdeFullWindow() throws IOException { final WindowedDataSegment windowedDataSegment = WindowedDataSegment.of(SEGMENT); final WindowedDataSegment roundTrip = MAPPER.readValue(MAPPER.writeValueAsBytes(windowedDataSegment), WindowedDataSegment.class); Assert.assertEquals(windowedDataSegment, roundTrip); Assert.assertEquals(SEGMENT, roundTrip.getSegment()); Assert.assertEquals(SEGMENT.getInterval(), roundTrip.getInterval()); }
public Map<String, Double> getLoadStatus() { // find available segments Map<String, Set<DataSegment>> availableSegments = Maps.newHashMap(); for (DataSegment dataSegment : getAvailableDataSegments()) { Set<DataSegment> segments = availableSegments.get(dataSegment.getDataSource()); if (segments == null) { segments = Sets.newHashSet(); availableSegments.put(dataSegment.getDataSource(), segments); } segments.add(dataSegment); } // find segments currently loaded Map<String, Set<DataSegment>> segmentsInCluster = Maps.newHashMap(); for (DruidServer druidServer : serverInventoryView.getInventory()) { for (DruidDataSource druidDataSource : druidServer.getDataSources()) { Set<DataSegment> segments = segmentsInCluster.get(druidDataSource.getName()); if (segments == null) { segments = Sets.newHashSet(); segmentsInCluster.put(druidDataSource.getName(), segments); } segments.addAll(druidDataSource.getSegments()); } } // compare available segments with currently loaded Map<String, Double> loadStatus = Maps.newHashMap(); for (Map.Entry<String, Set<DataSegment>> entry : availableSegments.entrySet()) { String dataSource = entry.getKey(); Set<DataSegment> segmentsAvailable = entry.getValue(); Set<DataSegment> loadedSegments = segmentsInCluster.get(dataSource); if (loadedSegments == null) { loadedSegments = Sets.newHashSet(); } Set<DataSegment> unloadedSegments = Sets.difference(segmentsAvailable, loadedSegments); loadStatus.put( dataSource, 100 * ((double) (segmentsAvailable.size() - unloadedSegments.size()) / (double) segmentsAvailable.size())); } return loadStatus; }
@Test(timeout = 4000L) public void testRealtimeIndexTask() throws Exception { monitorScheduler.addMonitor(EasyMock.anyObject(Monitor.class)); EasyMock.expectLastCall().atLeastOnce(); monitorScheduler.removeMonitor(EasyMock.anyObject(Monitor.class)); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(monitorScheduler, queryRunnerFactoryConglomerate); RealtimeIndexTask realtimeIndexTask = giveMeARealtimeIndexTask(); final String taskId = realtimeIndexTask.getId(); tq.add(realtimeIndexTask); // wait for task to process events and publish segment Assert.assertTrue(publishCountDown.await(1000, TimeUnit.MILLISECONDS)); // Realtime Task has published the segment, simulate loading of segment to a historical node so // that task finishes with SUCCESS status segmentCallbacks .get(0) .segmentAdded( new DruidServerMetadata("dummy", "dummy_host", 0, "historical", "dummy_tier", 0), mdc.getPublished().iterator().next()); // Wait for realtime index task to handle callback in plumber and succeed while (tsqa.getStatus(taskId).get().isRunnable()) { Thread.sleep(10); } Assert.assertTrue("Task should be in Success state", tsqa.getStatus(taskId).get().isSuccess()); Assert.assertEquals(1, announcedSinks); Assert.assertEquals(1, pushedSegments); Assert.assertEquals(1, mdc.getPublished().size()); DataSegment segment = mdc.getPublished().iterator().next(); Assert.assertEquals("test_ds", segment.getDataSource()); Assert.assertEquals(ImmutableList.of("dim1", "dim2"), segment.getDimensions()); Assert.assertEquals( new Interval(now.toString("YYYY-MM-dd") + "/" + now.plusDays(1).toString("YYYY-MM-dd")), segment.getInterval()); Assert.assertEquals(ImmutableList.of("count"), segment.getMetrics()); EasyMock.verify(monitorScheduler, queryRunnerFactoryConglomerate); }
protected void addSingleInventory(final DruidServer container, final DataSegment inventory) { log.info("Server[%s] added segment[%s]", container.getName(), inventory.getIdentifier()); if (container.getSegment(inventory.getIdentifier()) != null) { log.warn( "Not adding or running callbacks for existing segment[%s] on server[%s]", inventory.getIdentifier(), container.getName()); return; } container.addDataSegment(inventory.getIdentifier(), inventory); runSegmentCallbacks( new Function<SegmentCallback, CallbackAction>() { @Override public CallbackAction apply(SegmentCallback input) { return input.segmentAdded(container.getMetadata(), inventory); } }); }
@Override public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException { final Map<String, Object> loadSpec = segment.getLoadSpec(); final String region = MapUtils.getString(loadSpec, "region"); final String container = MapUtils.getString(loadSpec, "container"); final String path = MapUtils.getString(loadSpec, "path"); log.info("Pulling index at path[%s] to outDir[%s]", path, outDir); prepareOutDir(outDir); getSegmentFiles(region, container, path, outDir); }
private void serverAddedSegment(final DruidServerMetadata server, final DataSegment segment) { String segmentId = segment.getIdentifier(); synchronized (lock) { log.debug("Adding segment[%s] for server[%s]", segment, server); ServerSelector selector = selectors.get(segmentId); if (selector == null) { selector = new ServerSelector(segment, tierSelectorStrategy); VersionedIntervalTimeline<String, ServerSelector> timeline = timelines.get(segment.getDataSource()); if (timeline == null) { timeline = new VersionedIntervalTimeline<>(Ordering.natural()); timelines.put(segment.getDataSource(), timeline); } timeline.add( segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); selectors.put(segmentId, selector); } QueryableDruidServer queryableDruidServer = clients.get(server.getName()); if (queryableDruidServer == null) { queryableDruidServer = addServer(baseView.getInventoryValue(server.getName())); } selector.addServerAndUpdateSegment(queryableDruidServer, segment); } }
public Set<DataSegment> getAvailableDataSegments() { Set<DataSegment> availableSegments = Sets.newTreeSet(Comparators.inverse(DataSegment.bucketMonthComparator())); Iterable<DataSegment> dataSegments = Iterables.concat( Iterables.transform( databaseSegmentManager.getInventory(), new Function<DruidDataSource, Iterable<DataSegment>>() { @Override public Iterable<DataSegment> apply(DruidDataSource input) { return input.getSegments(); } })); for (DataSegment dataSegment : dataSegments) { if (dataSegment.getSize() < 0) { log.makeAlert("No size on Segment, wtf?").addData("segment", dataSegment).emit(); } availableSegments.add(dataSegment); } return availableSegments; }
public void addSegments(Iterable<DataSegment> segments) { try { final List<String> segmentFailures = Lists.newArrayList(); final List<DataSegment> validSegments = Lists.newArrayList(); for (DataSegment segment : segments) { log.info("Loading segment %s", segment.getIdentifier()); try { serverManager.loadSegment(segment); } catch (Exception e) { log.error(e, "Exception loading segment[%s]", segment.getIdentifier()); removeSegment(segment); segmentFailures.add(segment.getIdentifier()); continue; } File segmentInfoCacheFile = new File(config.getInfoDir(), segment.getIdentifier()); if (!segmentInfoCacheFile.exists()) { try { jsonMapper.writeValue(segmentInfoCacheFile, segment); } catch (IOException e) { log.error( e, "Failed to write to disk segment info cache file[%s]", segmentInfoCacheFile); removeSegment(segment); segmentFailures.add(segment.getIdentifier()); continue; } } validSegments.add(segment); } try { announcer.announceSegments(validSegments); } catch (IOException e) { throw new SegmentLoadingException(e, "Failed to announce segments[%s]", segments); } if (!segmentFailures.isEmpty()) { for (String segmentFailure : segmentFailures) { log.error("%s failed to load", segmentFailure); } throw new SegmentLoadingException( "%,d errors seen while loading segments", segmentFailures.size()); } } catch (SegmentLoadingException e) { log.makeAlert(e, "Failed to load segments for dataSource") .addData("segments", segments) .emit(); } }
@Test(expected = IOException.class) public void testupdateSegmentListThrowsExceptionWithUserSegmentsMismatch() throws Exception { PathSpec pathSpec = new DatasourcePathSpec( jsonMapper, null, new DatasourceIngestionSpec( testDatasource, testDatasourceInterval, null, ImmutableList.<DataSegment>of(SEGMENT.withVersion("v2")), null, null, null, null, false), null); testRunUpdateSegmentListIfDatasourcePathSpecIsUsed(pathSpec, testDatasourceInterval); }
private void serverRemovedSegment(DruidServerMetadata server, DataSegment segment) { String segmentId = segment.getIdentifier(); final ServerSelector selector; synchronized (lock) { log.debug("Removing segment[%s] from server[%s].", segmentId, server); selector = selectors.get(segmentId); if (selector == null) { log.warn("Told to remove non-existant segment[%s]", segmentId); return; } QueryableDruidServer queryableDruidServer = clients.get(server.getName()); if (!selector.removeServer(queryableDruidServer)) { log.warn( "Asked to disassociate non-existant association between server[%s] and segment[%s]", server, segmentId); } if (selector.isEmpty()) { VersionedIntervalTimeline<String, ServerSelector> timeline = timelines.get(segment.getDataSource()); selectors.remove(segmentId); final PartitionChunk<ServerSelector> removedPartition = timeline.remove( segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); if (removedPartition == null) { log.warn( "Asked to remove timeline entry[interval: %s, version: %s] that doesn't exist", segment.getInterval(), segment.getVersion()); } } } }
@Test public void testSimpleJob() throws IOException, InterruptedException { final SQLMetadataSegmentManager manager = new SQLMetadataSegmentManager( HadoopDruidConverterConfig.jsonMapper, new Supplier<MetadataSegmentManagerConfig>() { @Override public MetadataSegmentManagerConfig get() { return new MetadataSegmentManagerConfig(); } }, metadataStorageTablesConfigSupplier, connector); final List<DataSegment> oldSemgments = getDataSegments(manager); final File tmpDir = temporaryFolder.newFolder(); final HadoopConverterJob converterJob = new HadoopConverterJob( new HadoopDruidConverterConfig( DATASOURCE, interval, new IndexSpec(new RoaringBitmapSerdeFactory(), "uncompressed", "uncompressed"), oldSemgments, true, tmpDir.toURI(), ImmutableMap.<String, String>of(), null, tmpSegmentDir.toURI().toString())); final List<DataSegment> segments = Lists.newArrayList(converterJob.run()); Assert.assertNotNull("bad result", segments); Assert.assertEquals("wrong segment count", 4, segments.size()); Assert.assertTrue(converterJob.getLoadedBytes() > 0); Assert.assertTrue(converterJob.getWrittenBytes() > 0); Assert.assertTrue(converterJob.getWrittenBytes() > converterJob.getLoadedBytes()); Assert.assertEquals(oldSemgments.size(), segments.size()); final DataSegment segment = segments.get(0); Assert.assertTrue(interval.contains(segment.getInterval())); Assert.assertTrue(segment.getVersion().endsWith("_converted")); Assert.assertTrue(segment.getLoadSpec().get("path").toString().contains("_converted")); for (File file : tmpDir.listFiles()) { Assert.assertFalse(file.isDirectory()); Assert.assertTrue(file.isFile()); } final Comparator<DataSegment> segmentComparator = new Comparator<DataSegment>() { @Override public int compare(DataSegment o1, DataSegment o2) { return o1.getIdentifier().compareTo(o2.getIdentifier()); } }; Collections.sort(oldSemgments, segmentComparator); Collections.sort(segments, segmentComparator); for (int i = 0; i < oldSemgments.size(); ++i) { final DataSegment oldSegment = oldSemgments.get(i); final DataSegment newSegment = segments.get(i); Assert.assertEquals(oldSegment.getDataSource(), newSegment.getDataSource()); Assert.assertEquals(oldSegment.getInterval(), newSegment.getInterval()); Assert.assertEquals( Sets.<String>newHashSet(oldSegment.getMetrics()), Sets.<String>newHashSet(newSegment.getMetrics())); Assert.assertEquals( Sets.<String>newHashSet(oldSegment.getDimensions()), Sets.<String>newHashSet(newSegment.getDimensions())); Assert.assertEquals(oldSegment.getVersion() + "_converted", newSegment.getVersion()); Assert.assertTrue(oldSegment.getSize() < newSegment.getSize()); Assert.assertEquals(oldSegment.getBinaryVersion(), newSegment.getBinaryVersion()); } }
@Override public void getSegmentFiles(final DataSegment segment, final File outDir) throws SegmentLoadingException { final S3Coords s3Coords = new S3Coords(segment); log.info("Pulling index at path[%s] to outDir[%s]", s3Coords, outDir); if (!isObjectInBucket(s3Coords)) { throw new SegmentLoadingException("IndexFile[%s] does not exist.", s3Coords); } if (!outDir.exists()) { outDir.mkdirs(); } if (!outDir.isDirectory()) { throw new ISE("outDir[%s] must be a directory.", outDir); } try { S3Utils.retryS3Operation( new Callable<Void>() { @Override public Void call() throws Exception { long startTime = System.currentTimeMillis(); S3Object s3Obj = null; try { s3Obj = s3Client.getObject(s3Coords.bucket, s3Coords.path); try (InputStream in = s3Obj.getDataInputStream()) { final String key = s3Obj.getKey(); if (key.endsWith(".zip")) { CompressionUtils.unzip(in, outDir); } else if (key.endsWith(".gz")) { final File outFile = new File(outDir, toFilename(key, ".gz")); ByteStreams.copy( new GZIPInputStream(in), Files.newOutputStreamSupplier(outFile)); } else { ByteStreams.copy( in, Files.newOutputStreamSupplier(new File(outDir, toFilename(key, "")))); } log.info( "Pull of file[%s] completed in %,d millis", s3Obj, System.currentTimeMillis() - startTime); return null; } catch (IOException e) { throw new IOException( String.format("Problem decompressing object[%s]", s3Obj), e); } } finally { S3Utils.closeStreamsQuietly(s3Obj); } } }); } catch (Exception e) { try { FileUtils.deleteDirectory(outDir); } catch (IOException ioe) { log.warn( ioe, "Failed to remove output directory for segment[%s] after exception: %s", segment.getIdentifier(), outDir); } throw new SegmentLoadingException(e, e.getMessage()); } }
public static DataSegment serializeOutIndex( final DataSegment segmentTemplate, final Configuration configuration, final Progressable progressable, final TaskAttemptID taskAttemptID, final File mergedBase, final Path segmentBasePath) throws IOException { final FileSystem outputFS = FileSystem.get(segmentBasePath.toUri(), configuration); final Path tmpPath = new Path(segmentBasePath, String.format("index.zip.%d", taskAttemptID.getId())); final AtomicLong size = new AtomicLong(0L); final DataPusher zipPusher = (DataPusher) RetryProxy.create( DataPusher.class, new DataPusher() { @Override public long push() throws IOException { try (OutputStream outputStream = outputFS.create(tmpPath, true, DEFAULT_FS_BUFFER_SIZE, progressable)) { size.set(zipAndCopyDir(mergedBase, outputStream, progressable)); outputStream.flush(); } catch (IOException | RuntimeException exception) { log.error(exception, "Exception in retry loop"); throw exception; } return -1; } }, RetryPolicies.exponentialBackoffRetry( NUM_RETRIES, SECONDS_BETWEEN_RETRIES, TimeUnit.SECONDS)); zipPusher.push(); log.info("Zipped %,d bytes to [%s]", size.get(), tmpPath.toUri()); final Path finalIndexZipFilePath = new Path(segmentBasePath, "index.zip"); final URI indexOutURI = finalIndexZipFilePath.toUri(); final ImmutableMap<String, Object> loadSpec; // TODO: Make this a part of Pushers or Pullers switch (outputFS.getScheme()) { case "hdfs": loadSpec = ImmutableMap.<String, Object>of("type", "hdfs", "path", indexOutURI.toString()); break; case "s3": case "s3n": loadSpec = ImmutableMap.<String, Object>of( "type", "s3_zip", "bucket", indexOutURI.getHost(), "key", indexOutURI.getPath().substring(1) // remove the leading "/" ); break; case "file": loadSpec = ImmutableMap.<String, Object>of("type", "local", "path", indexOutURI.getPath()); break; default: throw new IAE("Unknown file system scheme [%s]", outputFS.getScheme()); } final DataSegment finalSegment = segmentTemplate .withLoadSpec(loadSpec) .withSize(size.get()) .withBinaryVersion(SegmentUtils.getVersionFromDir(mergedBase)); if (!renameIndexFiles(outputFS, tmpPath, finalIndexZipFilePath)) { throw new IOException( String.format( "Unable to rename [%s] to [%s]", tmpPath.toUri().toString(), finalIndexZipFilePath.toUri().toString())); } writeSegmentDescriptor( outputFS, finalSegment, new Path(segmentBasePath, "descriptor.json"), progressable); return finalSegment; }