Automatic compaction get stuck at the same time intervals

Hi guys,

I set up my Druid 0.13.0 cluster to ingest data every day with segment granularity DAY, this create time intervals of 1 day with 1 shard, with size ~100MB/day.

Then I set up the automatic compaction on the coordinator as per following:

{

“dataSource”: “DataProduction”,

“keepSegmentGranularity”: false,

“inputSegmentSizeBytes”: 576716800,

“targetCompactionSizeBytes”: 576716800,

“taskContext”: {

“priority”: 25

}

}

``

the coordinator issue the compaction of data between 25th and 30th of January and then between 20th and 25th of Jan. The tasks are successful but the new two time intervals (20 to 25, and 25 to 30) have two shards each, therefore at the next cycle the coordinator will issue the compaction for the same time intervals, hence the infinite loop.

What did I do wrong?

thank you!

Sergio

Hi,

I have a similar problem here https://groups.google.com/d/msg/druid-user/eDZuOA9KqKw/J7cwiqNSBAAJ

Hi,

In automatic compaction, the coordinator tries to optimize segments as much as it can. If the compacted segment size is not large enough (smaller than configured targetCompactionSizeBytes), the coordinator submits another compaction task to further optimize the compacted segment with others together. If new data is continuously being added to the same interval and the ingestion speed is faster than compaction speed, the automatic compaction might never check other intervals except the latest interval.

So, if you want to make auto compaction faster, you can raise the max task slots for compaction via the coordinator API (http://druid.io/docs/latest/operations/api-reference.html#compaction-configuration).

BTW, I would recommend to set “keepSegmentGranularity” to true. If it’s false, the automatic compaction would ignore the existing segment boundary. This option is deprecated and will be removed in the future release.

Jihoon

Hi Jihoon Son,

The problem we are facing is that the compaction task not making targetCompactionSizeBytes and result in an infinite loop.

For example, the compaction task tries to compact 2 segment shards with targetCompactionSizeBytes=500mb:

2019-03-06T04:00:00.000Z/2019-03-08T01:00:00.000Z_1 (230mb)

2019-03-06T04:00:00.000Z/2019-03-08T01:00:00.000Z_2 (230mb)

After the compaction task, those 2 segments shards size not compacted to 1 shard, they remain pretty much the same size, just with a new version. And the coordinator will try to compact those segment shard again and again without actually compact to single shard of 460mb.

Thanks.

Oh really?
It should be a bug if it does.

Do you see a log like Estimated targetPartitionSize[%d] = avgRowsPerByte[%f] * targetCompactionSizeBytes[%d] in the task log?

What are those values?

Jihoon

I think I know what goes wrong.

The compaction task will generate an internal index task. The targetPartionSize is calculated by targetCompactionSizeBytes and avgRowsPerByte.

There is another configuration, maxTotalRows for index task, the default value is 20000000. If targetPartionSize is larger than maxTotalRows, it won’t work as expected.

BTW, is there any way to change segment granularity for auto-compaction?

Nice, glad you figured it out!

Jihoon

BTW, is there any way to change segment granularity for auto-compaction?

This will be supported in 0.15.

Jihoon

Hi,

I am trying to overwrite the maxTotalRows with auto-compaction. My config is as follows:

{

“dataSource”: “druid_metrics”,

“inputSegmentSizeBytes”: 524288000,

“targetCompactionSizeBytes”: 524288000,

“skipOffsetFromLatest”: “PT3H”,

“keepSegmentGranularity”: false,

“tuningConfig”: {

“type”: “index”,

“maxTotalRows”: 100000000

}

}

However, the coordinator failed to post tasks to the overlord. Did I miss anything?

What is the error message of the coordinator?

Jihoon

The error message is like

2019-03-21T16:44:33,369 ERROR [Coordinator-Exec–0] org.apache.druid.server.coordinator.DruidCoordinator - Caught exception, ignoring so that schedule keeps going.: {class=org.apache.druid.server.coordinator.DruidCoordinator, exceptionType=class org.apache.druid.java.util

.common.ISE, exceptionMessage=Failed to post task[ClientCompactQuery{dataSource=‘druid_metrics’, segments=[DataSegment{size=142159181, shardSpec=NumberedShardSpec{partitionNum=0, partitions=0}, metrics=[count, value], dimensions=[service, metric, dataSource, host], versio

n=‘2019-03-21T08:14:31.426Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190319T120000.000Z_20190321T050000.000Z/2019-03-21T08_14_31.426Z/0_index.zip}, interval=2019-03-19T12:00:00.000Z/2019-03-21T05:00:00.000Z, dataSource='druid_metric

s’, binaryVersion=‘9’}, DataSegment{size=135440339, shardSpec=NumberedShardSpec{partitionNum=1, partitions=0}, metrics=[count, value], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T08:14:31.426Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/seg

ments-new/druid_metrics/20190319T120000.000Z_20190321T050000.000Z/2019-03-21T08_14_31.426Z/1_index.zip}, interval=2019-03-19T12:00:00.000Z/2019-03-21T05:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=30310704, shardSpec=NumberedShardSpec{part

itionNum=2, partitions=0}, metrics=[count, value], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T08:14:31.426Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190319T120000.000Z_20190321T050000.000Z/2019-03-21T08_14_3

1.426Z/2_index.zip}, interval=2019-03-19T12:00:00.000Z/2019-03-21T05:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=4950794, shardSpec=NumberedShardSpec{partitionNum=0, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataS

ource, host], version=‘2019-03-21T05:00:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190321T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/0_4651d3f1-a0ba-42ba-a5cf-4acbd785d17b_index.zip}, interval=2019-03-21T05:00:0

0.000Z/2019-03-21T06:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=3986029, shardSpec=NumberedShardSpec{partitionNum=1, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T05:00:00.034Z’

, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190321T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/1_9432d35f-1574-4455-83fe-01980cd0d958_index.zip}, interval=2019-03-21T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSource='dru

id_metrics’, binaryVersion=‘9’}, DataSegment{size=2218065, shardSpec=NumberedShardSpec{partitionNum=2, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T05:00:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/dr

uid/segments-new/druid_metrics/20190321T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/2_990fb937-a1a2-4415-b80b-89b2382229d9_index.zip}, interval=2019-03-21T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{siz

e=1253773, shardSpec=NumberedShardSpec{partitionNum=3, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T05:00:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190321T050000.0

00Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/3_63d4d981-369c-435c-b3cf-4e3972d6c386_index.zip}, interval=2019-03-21T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}], keepSegmentGranularity=false, targetCompactionSizeBytes=524288

000, tuningConfig=ClientCompactQueryTuningConfig{maxRowsInMemory='75000, maxTotalRows='20000000, indexSpec='IndexSpec{bitmapSerdeFactory=ConciseBitmapSerdeFactory{}, dimensionCompression=lz4, metricCompression=lz4, longEncoding=longs}, maxPendingPersists='0, publishTimeou

t='0}, context={priority=25}}]}

org.apache.druid.java.util.common.ISE: Failed to post task[ClientCompactQuery{dataSource=‘druid_metrics’, segments=[DataSegment{size=142159181, shardSpec=NumberedShardSpec{partitionNum=0, partitions=0}, metrics=[count, value], dimensions=[service, metric, dataSource, host

], version=‘2019-03-21T08:14:31.426Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190319T120000.000Z_20190321T050000.000Z/2019-03-21T08_14_31.426Z/0_index.zip}, interval=2019-03-19T12:00:00.000Z/2019-03-21T05:00:00.000Z, dataSource='dru

id_metrics’, binaryVersion=‘9’}, DataSegment{size=135440339, shardSpec=NumberedShardSpec{partitionNum=1, partitions=0}, metrics=[count, value], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T08:14:31.426Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/

druid/segments-new/druid_metrics/20190319T120000.000Z_20190321T050000.000Z/2019-03-21T08_14_31.426Z/1_index.zip}, interval=2019-03-19T12:00:00.000Z/2019-03-21T05:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=30310704, shardSpec=NumberedShard

Spec{partitionNum=2, partitions=0}, metrics=[count, value], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T08:14:31.426Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190319T120000.000Z_20190321T050000.000Z/2019-03-2

1T08_14_31.426Z/2_index.zip}, interval=2019-03-19T12:00:00.000Z/2019-03-21T05:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=4950794, shardSpec=NumberedShardSpec{partitionNum=0, partitions=0}, metrics=[value, count], dimensions=[service, metr

ic, dataSource, host], version=‘2019-03-21T05:00:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190321T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/0_4651d3f1-a0ba-42ba-a5cf-4acbd785d17b_index.zip}, interval=2019-03-2

1T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=3986029, shardSpec=NumberedShardSpec{partitionNum=1, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataSource, host], version='2019-03-21T05:00

:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190321T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/1_9432d35f-1574-4455-83fe-01980cd0d958_index.zip}, interval=2019-03-21T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSo

urce=‘druid_metrics’, binaryVersion=‘9’}, DataSegment{size=2218065, shardSpec=NumberedShardSpec{partitionNum=2, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T05:00:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://tl

0/user/druid/segments-new/druid_metrics/20190321T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/2_990fb937-a1a2-4415-b80b-89b2382229d9_index.zip}, interval=2019-03-21T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}, DataSe

gment{size=1253773, shardSpec=NumberedShardSpec{partitionNum=3, partitions=0}, metrics=[value, count], dimensions=[service, metric, dataSource, host], version=‘2019-03-21T05:00:00.034Z’, loadSpec={type=>hdfs, path=>hdfs://hdfscluster/user/druid/segments-new/druid_metrics/20190321

T050000.000Z_20190321T060000.000Z/2019-03-21T05_00_00.034Z/3_63d4d981-369c-435c-b3cf-4e3972d6c386_index.zip}, interval=2019-03-21T05:00:00.000Z/2019-03-21T06:00:00.000Z, dataSource=‘druid_metrics’, binaryVersion=‘9’}], keepSegmentGranularity=false, targetCompactionSizeByt

es=524288000, tuningConfig=ClientCompactQueryTuningConfig{maxRowsInMemory='75000, maxTotalRows='20000000, indexSpec='IndexSpec{bitmapSerdeFactory=ConciseBitmapSerdeFactory{}, dimensionCompression=lz4, metricCompression=lz4, longEncoding=longs},

maxPendingPersists='0, publishTimeout='0}, context={priority=25}}]

at org.apache.druid.client.indexing.HttpIndexingServiceClient.runTask(HttpIndexingServiceClient.java:135) ~[druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.client.indexing.HttpIndexingServiceClient.compactSegments(HttpIndexingServiceClient.java:111) ~[druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentCompactor.doRun(DruidCoordinatorSegmentCompactor.java:155) ~[druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentCompactor.run(DruidCoordinatorSegmentCompactor.java:102) ~[druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.server.coordinator.DruidCoordinator$CoordinatorRunnable.run(DruidCoordinator.java:677) [druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:573) [druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.server.coordinator.DruidCoordinator$2.call(DruidCoordinator.java:566) [druid-server-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at org.apache.druid.java.util.common.concurrent.ScheduledExecutors$2.run(ScheduledExecutors.java:93) [java-util-0.13.1-incubating-SNAPSHOT.jar:0.13.1-incubating-SNAPSHOT]

at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [?:1.8.0_181]

at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_181]

at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [?:1.8.0_181]

at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [?:1.8.0_181]

at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_181]

at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_181]

at java.lang.Thread.run(Thread.java:748) [?:1.8.0_181]

Hmm, this error means just the coordinator failed to post the task…
There should be a real error in the overlord log.

Would you please check again?

Jihoon

It looks a real bug. I’m fixing it at https://github.com/apache/incubator-druid/issues/7356.

Jihoon