Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Possible bug when loading multivalue+multipart String columns #7943

Open
jon-wei opened this issue Jun 21, 2019 · 0 comments
Open

Possible bug when loading multivalue+multipart String columns #7943

jon-wei opened this issue Jun 21, 2019 · 0 comments

Comments

@jon-wei
Copy link
Contributor

jon-wei commented Jun 21, 2019

Affected Version

0.13.0 and likely later versions, not sure what the earliest affected version is

Description

A user reported errors loading certain segments after upgrading from 0.11.0 -> 0.13.0: https://groups.google.com/forum/?pli=1#!topic/druid-user/m6IAMFLRrQM

The error and stack trace:

2019-06-12T17:42:46,230 ERROR [ZkCoordinator] org.apache.druid.server.coordination.SegmentLoadDropHandler - Failed to load segment for dataSource: {class=org.apache.druid.server.coordination.SegmentLoadDropHandler, exceptionType=class org.apache.druid.segment.loading.SegmentLoadingException, exceptionMessage=Exception loading segment[sapphire-stage-druid-metrics_2019-05-21T14:00:00.000Z_2019-05-21T15:00:00.000Z_2019-05-21T14:00:14.673Z], segment=DataSegment{size=7112133889, shardSpec=NumberedShardSpec{partitionNum=0, partitions=0}, metrics=[count, value_sum, value_min, value_max], dimensions=[feed, service, host, version, metric, dataSource, duration, hasFilters, id, interval, segment, type, clusterName, memKind, poolKind, poolName, bufferpoolName, gcGen, gcName, gcGenSpaceName, context, remoteAddress, success, server, taskId, taskType, tier, priority, taskStatus], version='2019-05-21T14:00:14.673Z', loadSpec={type=>hdfs, path=>hdfs://xxxxx/druid/sapphire-stage/data/sapphire-stage-druid-metrics/20190521T140000.000Z_20190521T150000.000Z/2019-05-21T14_00_14.673Z/0_index.zip}, interval=2019-05-21T14:00:00.000Z/2019-05-21T15:00:00.000Z, dataSource='sapphire-stage-druid-metrics', binaryVersion='9'}}
org.apache.druid.segment.loading.SegmentLoadingException: Exception loading segment[sapphire-stage-druid-metrics_2019-05-21T14:00:00.000Z_2019-05-21T15:00:00.000Z_2019-05-21T14:00:14.673Z]
       at org.apache.druid.server.coordination.SegmentLoadDropHandler.loadSegment(SegmentLoadDropHandler.java:265) ~[druid-server-0.13.0.jar:0.13.0]
       at org.apache.druid.server.coordination.SegmentLoadDropHandler.addSegment(SegmentLoadDropHandler.java:307) [druid-server-0.13.0.jar:0.13.0]
       at org.apache.druid.server.coordination.SegmentChangeRequestLoad.go(SegmentChangeRequestLoad.java:47) [druid-server-0.13.0.jar:0.13.0]
       at org.apache.druid.server.coordination.ZkCoordinator$1.childEvent(ZkCoordinator.java:118) [druid-server-0.13.0.jar:0.13.0]
       at org.apache.curator.framework.recipes.cache.PathChildrenCache$5.apply(PathChildrenCache.java:520) [curator-recipes-4.0.0.jar:4.0.0]
       at org.apache.curator.framework.recipes.cache.PathChildrenCache$5.apply(PathChildrenCache.java:514) [curator-recipes-4.0.0.jar:4.0.0]
       at org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93) [curator-framework-4.0.0.jar:4.0.0]
       at org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$DirectExecutorService.execute(MoreExecutors.java:296) [curator-client-4.0.0.jar:?]
       at org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85) [curator-framework-4.0.0.jar:4.0.0]
       at org.apache.curator.framework.recipes.cache.PathChildrenCache.callListeners(PathChildrenCache.java:512) [curator-recipes-4.0.0.jar:4.0.0]
       at org.apache.curator.framework.recipes.cache.EventOperation.invoke(EventOperation.java:35) [curator-recipes-4.0.0.jar:4.0.0]
       at org.apache.curator.framework.recipes.cache.PathChildrenCache$9.run(PathChildrenCache.java:771) [curator-recipes-4.0.0.jar:4.0.0]
       at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_73]
       at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_73]
       at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_73]
       at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_73]
       at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [?:1.8.0_73]
       at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [?:1.8.0_73]
       at java.lang.Thread.run(Thread.java:745) [?:1.8.0_73]
Caused by: org.apache.druid.java.util.common.IAE: use read(ByteBuffer buffer, ObjectStrategy<T> strategy, SmooshedFileMapper fileMapper) to read version 2 indexed.
       at org.apache.druid.segment.data.GenericIndexed.read(GenericIndexed.java:131) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier.fromByteBuffer(CompressedVSizeColumnarIntsSupplier.java:161) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.data.V3CompressedVSizeColumnarMultiIntsSupplier.fromByteBuffer(V3CompressedVSizeColumnarMultiIntsSupplier.java:67) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.serde.DictionaryEncodedColumnPartSerde$1.readMultiValuedColumn(DictionaryEncodedColumnPartSerde.java:381) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.serde.DictionaryEncodedColumnPartSerde$1.read(DictionaryEncodedColumnPartSerde.java:309) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.column.ColumnDescriptor.read(ColumnDescriptor.java:106) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.IndexIO$V9IndexLoader.deserializeColumn(IndexIO.java:618) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.IndexIO$V9IndexLoader.load(IndexIO.java:593) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.IndexIO.loadIndex(IndexIO.java:187) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory.factorize(MMappedQueryableSegmentizerFactory.java:48) ~[druid-processing-0.13.0.jar:0.13.0]
       at org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager.getSegment(SegmentLoaderLocalCacheManager.java:123) ~[druid-server-0.13.0.jar:0.13.0]
       at org.apache.druid.server.SegmentManager.getAdapter(SegmentManager.java:196) ~[druid-server-0.13.0.jar:0.13.0]
       at org.apache.druid.server.SegmentManager.loadSegment(SegmentManager.java:157) ~[druid-server-0.13.0.jar:0.13.0]
       at org.apache.druid.server.coordination.SegmentLoadDropHandler.loadSegment(SegmentLoadDropHandler.java:261) ~[druid-server-0.13.0.jar:0.13.0]
       ... 18 more

The segment in question is quite large (7GB+): DataSegment{size=7112133889,

From that, it looks like CompressedVSizeColumnarIntsSupplier.fromByteBuffer may need to handle the multipart column case and sometimes call public static <T> GenericIndexed<T> read(ByteBuffer buffer, ObjectStrategy<T> strategy, SmooshedFileMapper fileMapper) with a SmooshedFileMapper.

  public static CompressedVSizeColumnarIntsSupplier fromByteBuffer(
      ByteBuffer buffer,
      ByteOrder order
  )
  {
    byte versionFromBuffer = buffer.get();

    if (versionFromBuffer == VERSION) {
      final int numBytes = buffer.get();
      final int totalSize = buffer.getInt();
      final int sizePer = buffer.getInt();

      final CompressionStrategy compression = CompressionStrategy.forId(buffer.get());

      return new CompressedVSizeColumnarIntsSupplier(
          totalSize,
          sizePer,
          numBytes,
          GenericIndexed.read(buffer, new DecompressingByteBufferObjectStrategy(order, compression)),
          compression
      );

    }

    throw new IAE("Unknown version[%s]", versionFromBuffer);
  }
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

No branches or pull requests

1 participant