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);
    }
  }
  @Test
  public void testV1Serialization() throws Exception {

    final Interval interval = new Interval("2011-10-01/2011-10-02");
    final ImmutableMap<String, Object> loadSpec =
        ImmutableMap.<String, Object>of("something", "or_other");

    DataSegment segment =
        new DataSegment(
            "something",
            interval,
            "1",
            loadSpec,
            Arrays.asList("dim1", "dim2"),
            Arrays.asList("met1", "met2"),
            NoneShardSpec.instance(),
            IndexIO.CURRENT_VERSION_ID,
            1);

    final Map<String, Object> objectMap =
        mapper.readValue(
            mapper.writeValueAsString(segment), new TypeReference<Map<String, Object>>() {});

    Assert.assertEquals(10, objectMap.size());
    Assert.assertEquals("something", objectMap.get("dataSource"));
    Assert.assertEquals(interval.toString(), objectMap.get("interval"));
    Assert.assertEquals("1", objectMap.get("version"));
    Assert.assertEquals(loadSpec, objectMap.get("loadSpec"));
    Assert.assertEquals("dim1,dim2", objectMap.get("dimensions"));
    Assert.assertEquals("met1,met2", objectMap.get("metrics"));
    Assert.assertEquals(ImmutableMap.of("type", "none"), objectMap.get("shardSpec"));
    Assert.assertEquals(IndexIO.CURRENT_VERSION_ID, objectMap.get("binaryVersion"));
    Assert.assertEquals(1, objectMap.get("size"));

    DataSegment deserializedSegment =
        mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);

    Assert.assertEquals(segment.getDataSource(), deserializedSegment.getDataSource());
    Assert.assertEquals(segment.getInterval(), deserializedSegment.getInterval());
    Assert.assertEquals(segment.getVersion(), deserializedSegment.getVersion());
    Assert.assertEquals(segment.getLoadSpec(), deserializedSegment.getLoadSpec());
    Assert.assertEquals(segment.getDimensions(), deserializedSegment.getDimensions());
    Assert.assertEquals(segment.getMetrics(), deserializedSegment.getMetrics());
    Assert.assertEquals(segment.getShardSpec(), deserializedSegment.getShardSpec());
    Assert.assertEquals(segment.getSize(), deserializedSegment.getSize());
    Assert.assertEquals(segment.getIdentifier(), deserializedSegment.getIdentifier());

    deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
    Assert.assertEquals(0, segment.compareTo(deserializedSegment));

    deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
    Assert.assertEquals(0, deserializedSegment.compareTo(segment));

    deserializedSegment = mapper.readValue(mapper.writeValueAsString(segment), DataSegment.class);
    Assert.assertEquals(segment.hashCode(), deserializedSegment.hashCode());
  }
 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())));
   }
 }
Exemple #4
0
  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;
  }
Exemple #5
0
    @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;
    }
  @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);
  }
  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());
    }
  }
Exemple #9
0
 public void removeSegment(DataSegment segment) {
   log.info("Removing Segment[%s]", segment);
   databaseSegmentManager.removeSegment(segment.getDataSource(), segment.getIdentifier());
 }