Skip to content

Historical unloads damaged segments automatically when lazy on start.#10688

Merged
clintropolis merged 34 commits intoapache:masterfrom
zhangyue19921010:historical-lazyOnStart-with-fileCheck
Jan 17, 2021
Merged

Historical unloads damaged segments automatically when lazy on start.#10688
clintropolis merged 34 commits intoapache:masterfrom
zhangyue19921010:historical-lazyOnStart-with-fileCheck

Conversation

@zhangyue19921010
Copy link
Copy Markdown
Contributor

@zhangyue19921010 zhangyue19921010 commented Dec 17, 2020

Description

The feature mentioned in #6988 is very useful and can greatly improve the speed of historical node restart. But there are some potential risks due to push the loading work to query time.

If segment files like meta.smoosh or data.smoosh which Historical is served already are damaged, Historical with lazyOnStart enable can restart successfully but will not detect file corruption during restarting. When queries hit these damaged files, these queries will fail and will not expose which specific files are damaged like mentioned in #10608

Tests

Here are the undamaged segment files.

屏幕快照 2020-12-17 下午6 29 49

First case : meta.smoosh is damaged, for example

Remove a line video_cro_network_name,0,997718245,1000412345 in that meta.smoosh file.

屏幕快照 2020-12-17 下午6 33 27

Restart Historical node with lazy on start enable.

Second case : data.smoosh is damaged, for example

Delete several bytes in that 00000.smoosh file randomly.
Restart Historical node with lazy on start enable.

In all scenarios, Historical can restart successfully, but queries will failed.

2020-11-26T10:56:14,031 INFO [DruidSchema-Cache-0] org.apache.druid.server.log.LoggingRequestLogger - 2020-11-26T10:56:14.026Z		{"queryType":"segmentMetadata","dataSource":{"type":"table","name":"traffic__ops_feed__realtime__second__dev__lazy__test"},"intervals":{"type":"segments","segments":[{"itvl":"2020-11-26T03:00:00.000Z/2020-11-26T04:00:00.000Z","ver":"2020-11-26T03:30:17.710Z","part":0},{"itvl":"2020-11-26T03:00:00.000Z/2020-11-26T04:00:00.000Z","ver":"2020-11-26T03:30:17.710Z","part":1},{"itvl":"2020-11-26T03:00:00.000Z/2020-11-26T04:00:00.000Z","ver":"2020-11-26T03:30:17.710Z","part":2},{"itvl":"2020-11-26T03:00:00.000Z/2020-11-26T04:00:00.000Z","ver":"2020-11-26T03:30:17.710Z","part":3},{"itvl":"2020-11-26T03:00:00.000Z/2020-11-26T04:00:00.000Z","ver":"2020-11-26T03:30:17.710Z","part":4},{"itvl":"2020-11-26T03:00:00.000Z/2020-11-26T04:00:00.000Z","ver":"2020-11-26T03:30:17.710Z","part":5}]},"toInclude":{"type":"all"},"merge":false,"context":{"queryId":"a022fbbf-d2fa-408b-bd21-698645c74f62"},"analysisTypes":[],"usingDefaultInterval":false,"lenientAggregatorMerge":false,"descending":false,"granularity":{"type":"all"}}	{"query/time":4,"query/bytes":-1,"success":false,"identity":"allowAll","exception":"java.lang.RuntimeException: com.fasterxml.jackson.core.io.JsonEOFException: Unexpected end-of-input: expected close marker for Array (start marker at [Source: (SequenceInputStream); line: -1, column: -1])\n at [Source: (SequenceInputStream); line: -1, column: 8041]"}
2020-11-26T10:56:14,031 WARN [DruidSchema-Cache-0] org.apache.druid.sql.calcite.schema.DruidSchema - Metadata refresh failed, trying again soon.
java.lang.RuntimeException: com.fasterxml.jackson.core.io.JsonEOFException: Unexpected end-of-input: expected close marker for Array (start marker at [Source: (SequenceInputStream); line: -1, column: -1])
 at [Source: (SequenceInputStream); line: -1, column: 8041]
	at org.apache.druid.client.JsonParserIterator.next(JsonParserIterator.java:119) ~[druid-server-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.BaseSequence.makeYielder(BaseSequence.java:90) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.BaseSequence.toYielder(BaseSequence.java:69) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.MappedSequence.toYielder(MappedSequence.java:49) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$ResultBatch.fromSequence(ParallelMergeCombiningSequence.java:847) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$SequenceBatcher.block(ParallelMergeCombiningSequence.java:897) ~[druid-core-0.17.1.jar:0.17.1]
	at java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3313) ~[?:1.8.0_221]
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$SequenceBatcher.getBatchYielder(ParallelMergeCombiningSequence.java:886) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$YielderBatchedResultsCursor.initialize(ParallelMergeCombiningSequence.java:993) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.ParallelMergeCombiningSequence$PrepareMergeCombineInputsAction.compute(ParallelMergeCombiningSequence.java:702) ~[druid-core-0.17.1.jar:0.17.1]
	at java.util.concurrent.RecursiveAction.exec(RecursiveAction.java:189) ~[?:1.8.0_221]
	at java.util.concurrent.ForkJoinTask.doExec(ForkJoinTask.java:289) ~[?:1.8.0_221]
	at java.util.concurrent.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1056) ~[?:1.8.0_221]
	at java.util.concurrent.ForkJoinPool.runWorker(ForkJoinPool.java:1692) ~[?:1.8.0_221]
	at java.util.concurrent.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:157) ~[?:1.8.0_221]
Caused by: com.fasterxml.jackson.core.io.JsonEOFException: Unexpected end-of-input: expected close marker for Array (start marker at [Source: (SequenceInputStream); line: -1, column: -1])
 at [Source: (SequenceInputStream); line: -1, column: 8041]
	at com.fasterxml.jackson.core.base.ParserMinimalBase._reportInvalidEOF(ParserMinimalBase.java:664) ~[jackson-core-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.dataformat.smile.SmileParserBase._handleEOF(SmileParserBase.java:746) ~[jackson-dataformat-smile-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.dataformat.smile.SmileParser._eofAsNextToken(SmileParser.java:2691) ~[jackson-dataformat-smile-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.dataformat.smile.SmileParser.nextToken(SmileParser.java:378) ~[jackson-dataformat-smile-2.10.1.jar:2.10.1]
	at org.apache.druid.client.JsonParserIterator.next(JsonParserIterator.java:115) ~[druid-server-0.17.1.jar:0.17.1]
	... 14 more

Solution

This PR make Historical with lazy on start have the ability that:
Failed queries caused by damaged segment files will trigger unload action automatically. After the unload, the coordinator will ask another historical to load this segment.
Through this PR, the system heals itself and the administrator need to do nothing.


This PR has:

  • been self-reviewed.
  • added documentation for new or modified features or behaviors.
  • added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
  • added or updated version, license, or notice information in licenses.yaml
  • added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
  • added unit tests or modified existing tests to cover new code paths, ensuring the threshold for code coverage is met.
  • added integration tests.
  • been tested in a test Druid cluster.

Key changed/added classes in this PR
  • IndexIO.java

@kaijianding
Copy link
Copy Markdown
Contributor

How about trigger unload or force delete when the query failed due to segment corruption? After the unload, the coordinator will ask another historical to load this segment.
In this way, the system heals itself and the administrator need to do nothing

@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

zhangyue19921010 commented Dec 21, 2020

Hi @kaijianding. As you can see, there are few error messages when query failed due to segments damaged. So that it is hard to distinguish what kind of exception is caused by damaged segments, wrong query or others and find out which specific files are damaged. Because a query usually reaches multiple segments. What's more, let query failure trigger a delete segment action seems too dangerous in my opinion.

@kaijianding
Copy link
Copy Markdown
Contributor

@zhangyue19921010 I guess the exception from loadIndex can be a signal of corruption. When you distinguish the corruption exception, it is safe to unload this segment and let the coordinator choose another historical to load(usually, you have replicas). Compare to restart behavior, I think the unloading behavior is more efficient and no human involvement is needed.

@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

zhangyue19921010 commented Dec 22, 2020

Hi @kaijianding. This PR is just aiming to let druid regain the ability which is broken by lazyOnStart to check the availability of all segments and reload corrupted segments during historical restart.

Is what you proposed that make druid have the ability to find and solve corrupted segments during runtime no matter lazyOnStart or not?

It is actually a nice idea which can enhance the robustness of Druid and solve the corrupted segments problems once and for all. I will keep digging maybe make an another PR to make it happen if I understand right. What do you think? :)

@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

zhangyue19921010 commented Dec 22, 2020

@zhangyue19921010 I guess the exception from loadIndex can be a signal of corruption. When you distinguish the corruption exception, it is safe to unload this segment and let the coordinator choose another historical to load(usually, you have replicas). Compare to restart behavior, I think the unloading behavior is more efficient and no human involvement is needed.

Unfortunately there is no extra exception in loadIndex because of lazy action, and historical will restart successfully. what I am trying to do is that do segments file check and throws IOException in loadIndex :)

@kaijianding
Copy link
Copy Markdown
Contributor

kaijianding commented Dec 22, 2020

Is what you proposed that make druid have the ability to find and solve corrupted segments during runtime no matter lazyOnStart or not?

Only when lazy load should be good enough.

   if (lazy) {
      columns.put(columnName, Suppliers.memoize(
          () -> {
            try {
              return deserializeColumn(mapper, colBuffer, smooshedFiles);
            }
            catch (IOException e) {
              throw Throwables.propagate(e);
            }
          }
      ));
    }

Maybe do something in this IOException catch and trigger SegmentManager.dropSegment(dataSegment) somehow.
U need pass in the segmentManager instance in some way. @zhangyue19921010

@zhangyue19921010 zhangyue19921010 force-pushed the historical-lazyOnStart-with-fileCheck branch from 46303d4 to 7775658 Compare January 5, 2021 04:38
@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

zhangyue19921010 commented Jan 5, 2021

Hi @kaijianding , sorry for late response. I just change the strategy for lazyOnStart action as you suggested.
I build a new CallBack action () -> this.removeSegment(segment, DataSegmentChangeCallback.NOOP, false). When query failed caused by lazyOnStart, this action will unload this segment and let the coordinator choose another historical to load. I have tested it on dev cluster and works fine. Could you please help review the code? Thanks.

@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

Here is the full logs during query failed and trigger unload action.

2021-01-05T03:19:42,622 INFO [ZKCoordinator--2] org.apache.druid.server.coordination.ZkCoordinator - Completed request [LOAD: todate_ad_historical_hc_regtest_2021-01-04T00:00:00.000Z_2021-01-05T00:00:00.000Z_2021-01-05T02:57:11.749Z_1]
2021-01-05T03:19:42,622 INFO [ZkCoordinator] org.apache.druid.server.coordination.ZkCoordinator - zNode[/druid/loadQueue/druid-dev-8-historical-0.druid-dev-8-historical.druid-dev-8.svc.cluster.local:8083/todate_ad_historical_hc_regtest_2021-01-04T00:00:00.000Z_2021-01-05T00:00:00.000Z_2021-01-05T02:57:11.749Z_1] was removed
2021-01-05T03:21:14,592 WARN [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.segment.IndexIO - Exception when deserialize Column country
com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot construct instance of `org.apache.druid.segment.column.ColumnDescriptor` (although at least one Creator exists): no String-argument constructor/factory method to deserialize from String value ('valueType')
 at [Source: (StringReader); line: 1, column: 1]
	at com.fasterxml.jackson.databind.exc.MismatchedInputException.from(MismatchedInputException.java:63) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.DeserializationContext.reportInputMismatch(DeserializationContext.java:1429) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.DeserializationContext.handleMissingInstantiator(DeserializationContext.java:1059) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.deser.ValueInstantiator._createFromStringFallbacks(ValueInstantiator.java:371) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.deser.std.StdValueInstantiator.createFromString(StdValueInstantiator.java:323) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromString(BeanDeserializerBase.java:1373) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeOther(BeanDeserializer.java:171) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.deser.BeanDeserializer.deserialize(BeanDeserializer.java:161) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.ObjectMapper._readMapAndClose(ObjectMapper.java:4202) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3205) ~[jackson-databind-2.10.1.jar:2.10.1]
	at com.fasterxml.jackson.databind.ObjectMapper.readValue(ObjectMapper.java:3173) ~[jackson-databind-2.10.1.jar:2.10.1]
	at org.apache.druid.segment.IndexIO$V9IndexLoader.deserializeColumn(IndexIO.java:657) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.segment.IndexIO$V9IndexLoader.lambda$load$0(IndexIO.java:603) ~[druid-processing-0.17.1.jar:0.17.1]
	at com.google.common.base.Suppliers$MemoizingSupplier.get(Suppliers.java:125) ~[guava-16.0.1.jar:?]
	at org.apache.druid.segment.SimpleQueryableIndex.getColumnHolder(SimpleQueryableIndex.java:163) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentAnalyzer.analyze(SegmentAnalyzer.java:103) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory$1.run(SegmentMetadataQueryRunnerFactory.java:92) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.ReferenceCountingSegmentQueryRunner.run(ReferenceCountingSegmentQueryRunner.java:51) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.MetricsEmittingQueryRunner.lambda$run$0(MetricsEmittingQueryRunner.java:97) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.LazySequence.accumulate(LazySequence.java:40) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.SequenceWrapper.wrap(SequenceWrapper.java:55) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.MappedSequence.accumulate(MappedSequence.java:43) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.SequenceWrapper.wrap(SequenceWrapper.java:55) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.LazySequence.accumulate(LazySequence.java:40) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.SequenceWrapper.wrap(SequenceWrapper.java:55) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner$1.accumulate(SpecificSegmentQueryRunner.java:79) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner.doNamed(SpecificSegmentQueryRunner.java:163) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner.access$100(SpecificSegmentQueryRunner.java:42) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner$2.wrap(SpecificSegmentQueryRunner.java:145) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.CPUTimeMetricQueryRunner$1.wrap(CPUTimeMetricQueryRunner.java:74) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.Sequence.toList(Sequence.java:85) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory$2$1$1.call(SegmentMetadataQueryRunnerFactory.java:217) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory$2$1$1.call(SegmentMetadataQueryRunnerFactory.java:213) [druid-processing-0.17.1.jar:0.17.1]
	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_221]
	at org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:247) [druid-processing-0.17.1.jar:0.17.1]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_221]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_221]
	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_221]
2021-01-05T03:21:14,595 INFO [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.server.coordination.BatchDataSegmentAnnouncer - Unannouncing segment[traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z] at path[/druid/segments/druid-dev-8-historical-0.druid-dev-8-historical.druid-dev-8.svc.cluster.local:8083/druid-dev-8-historical-0.druid-dev-8-historical.druid-dev-8.svc.cluster.local:8083_historical__default_tier_2021-01-05T03:09:36.189Z_bc91ea1098a749438ad4654b430c5cf61474]
2021-01-05T03:21:14,599 INFO [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.server.SegmentManager - Attempting to close segment traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z
2021-01-05T03:21:14,599 INFO [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager - Deleting directory[/var/druid/segment-cache/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0]
2021-01-05T03:21:14,601 WARN [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.segment.loading.StorageLocation - SegmentDir[/var/druid/segment-cache/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0] is not found under this location[/var/druid/segment-cache]
2021-01-05T03:21:14,748 WARN [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.segment.IndexIO - Exception when deserialize Column country
java.nio.BufferUnderflowException: null
	at java.nio.DirectByteBuffer.get(DirectByteBuffer.java:271) ~[?:1.8.0_221]
	at java.nio.ByteBuffer.get(ByteBuffer.java:715) ~[?:1.8.0_221]
	at org.apache.druid.common.utils.SerializerUtils.readBytes(SerializerUtils.java:68) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.common.utils.SerializerUtils.readString(SerializerUtils.java:62) ~[druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.segment.IndexIO$V9IndexLoader.deserializeColumn(IndexIO.java:658) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.segment.IndexIO$V9IndexLoader.lambda$load$0(IndexIO.java:603) ~[druid-processing-0.17.1.jar:0.17.1]
	at com.google.common.base.Suppliers$MemoizingSupplier.get(Suppliers.java:125) ~[guava-16.0.1.jar:?]
	at org.apache.druid.segment.SimpleQueryableIndex.getColumnHolder(SimpleQueryableIndex.java:163) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentAnalyzer.analyze(SegmentAnalyzer.java:103) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory$1.run(SegmentMetadataQueryRunnerFactory.java:92) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.ReferenceCountingSegmentQueryRunner.run(ReferenceCountingSegmentQueryRunner.java:51) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.MetricsEmittingQueryRunner.lambda$run$0(MetricsEmittingQueryRunner.java:97) ~[druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.LazySequence.accumulate(LazySequence.java:40) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.SequenceWrapper.wrap(SequenceWrapper.java:55) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.MappedSequence.accumulate(MappedSequence.java:43) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.SequenceWrapper.wrap(SequenceWrapper.java:55) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.LazySequence.accumulate(LazySequence.java:40) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.SequenceWrapper.wrap(SequenceWrapper.java:55) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner$1.accumulate(SpecificSegmentQueryRunner.java:79) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner.doNamed(SpecificSegmentQueryRunner.java:163) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner.access$100(SpecificSegmentQueryRunner.java:42) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.spec.SpecificSegmentQueryRunner$2.wrap(SpecificSegmentQueryRunner.java:145) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence$1.get(WrappingSequence.java:50) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.CPUTimeMetricQueryRunner$1.wrap(CPUTimeMetricQueryRunner.java:74) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.WrappingSequence.accumulate(WrappingSequence.java:45) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.java.util.common.guava.Sequence.toList(Sequence.java:85) [druid-core-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory$2$1$1.call(SegmentMetadataQueryRunnerFactory.java:217) [druid-processing-0.17.1.jar:0.17.1]
	at org.apache.druid.query.metadata.SegmentMetadataQueryRunnerFactory$2$1$1.call(SegmentMetadataQueryRunnerFactory.java:213) [druid-processing-0.17.1.jar:0.17.1]
	at java.util.concurrent.FutureTask.run(FutureTask.java:266) [?:1.8.0_221]
	at org.apache.druid.query.PrioritizedListenableFutureTask.run(PrioritizedExecutorService.java:247) [druid-processing-0.17.1.jar:0.17.1]
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) [?:1.8.0_221]
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) [?:1.8.0_221]
	at java.lang.Thread.run(Thread.java:748) [?:1.8.0_221]
2021-01-05T03:21:14,756 WARN [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.server.coordination.BatchDataSegmentAnnouncer - No path to unannounce segment[traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z]
2021-01-05T03:21:14,756 INFO [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.server.SegmentManager - Told to delete a queryable on dataSource[traffic__ops_feed__realtime__second__dev__big__segment] for interval[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z] and version[2020-12-02T02:33:58.519Z] that I don't have.
2021-01-05T03:21:14,760 WARN [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.segment.loading.SegmentLoaderLocalCacheManager - Asked to cleanup something[DataSegment{binaryVersion=9, id=traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z, loadSpec={type=>s3_zip, bucket=>pqm-druid-dev, key=>rtstorage/segments/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0/index.zip, S3Schema=>s3n}, dimensions=[video_cro_network_id, video_cro_network_name, distributor_network_id, distributor_network_name, profile_id, profile_name, is_active_device, is_filtered, service_type, platform, ad_unit_type, country, country_name, state, state_name, dma, dma_name, syscode, syscode_name, tv_network_id, tv_network_name, linear_campaign_type, spot_type], metrics=[ack_ad_click, ack_ad_complete, ack_ad_first_quartile, ack_ad_impression, ack_ad_mid_point, ack_ad_third_quartile, ack_err_adm_e_3p_comp, ack_err_adm_e_device_limit, ack_err_adm_e_io, ack_err_adm_e_no_ad, ack_err_adm_e_no_render, ack_err_adm_e_parse, ack_err_adm_e_render_init, ack_err_adm_e_security, ack_err_adm_e_timeout, ack_err_adm_e_unknown, ack_err_psn_abnormal_termination_of_playout, ack_err_psn_asset_info_invalid, ack_err_psn_bit_rate_mismatch, ack_err_psn_insertion_point_time_exceeded, ack_err_psn_message_validation_failed, ack_err_psn_timeout, ack_err_psn_unknown_message_reference, ack_err_vast_100, ack_err_vast_202, ack_err_vast_300, ack_err_vast_301, ack_err_vast_302, ack_err_vast_303, ack_err_vast_400, ack_err_vast_402, ack_err_vast_403, ack_err_vast_900, ad_delivered_ad, ad_delivered_ad_fallback, ad_delivered_ad_primary, ad_err_fallback_to_evergreen, ad_err_full_avail_no_variant_segment, ad_err_inactive_addressable_order, count, req_ad_request, req_ad_request_with_midroll_slot, req_ad_request_with_video_slot, req_empty_response, req_empty_response_with_midroll_slot, req_empty_response_with_video_slot, req_err_break_duration_invalid, req_err_break_no_schedule_ad, req_err_no_mac_address, req_err_no_profile, req_err_no_signal_id, req_err_schedule_creative_validation_failed, req_err_schedule_not_found, req_err_signal_no_bind_break, req_err_station_not_found, req_err_syscode_not_found, req_resp_time_gt_1500ms, req_resp_time_lt_100ms, req_resp_time_lt_1500ms, req_resp_time_lt_300ms, req_resp_time_lt_500ms, slot_avails, slot_unfilled_avails], shardSpec=NumberedShardSpec{partitionNum=0, partitions=0}, lastCompactionState=null, size=3388721170}] that didn't exist.  Skipping.
2021-01-05T03:21:14,761 WARN [segmentMetadata_traffic__ops_feed__realtime__second__dev__big__segment_[2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z]] org.apache.druid.server.coordination.SegmentLoadDropHandler - Unable to delete segmentInfoCacheFile[/var/druid/segment-cache/info_dir/traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z]
2021-01-05T03:21:14,761 ERROR [qtp450476243-159[segmentMetadata_[traffic__ops_feed__realtime__second__dev__big__segment]_7571e94d-4e59-4486-bc0f-bd4d447d2e18]] org.apache.druid.server.QueryResource - Exception handling request: {class=org.apache.druid.server.QueryResource, exceptionType=class java.lang.RuntimeException, exceptionMessage=java.util.concurrent.ExecutionException: java.lang.RuntimeException: com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot construct instance of `org.apache.druid.segment.column.ColumnDescriptor` (although at least one Creator exists): no String-argument constructor/factory method to deserialize from String value ('valueType')
 at [Source: (StringReader); line: 1, column: 1], query={"queryType":"segmentMetadata","dataSource":{"type":"table","name":"traffic__ops_feed__realtime__second__dev__big__segment"},"intervals":{"type":"segments","segments":[{"itvl":"2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z","ver":"2020-12-02T02:33:58.519Z","part":0},{"itvl":"2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z","ver":"2020-12-02T02:33:58.519Z","part":1}]},"toInclude":{"type":"all"},"merge":false,"context":{"defaultTimeout":60000,"finalize":false,"maxQueuedBytes":50000000,"maxScatterGatherBytes":9223372036854775807,"queryFailTime":1609816934586,"queryId":"7571e94d-4e59-4486-bc0f-bd4d447d2e18","timeout":60000},"analysisTypes":[],"usingDefaultInterval":false,"lenientAggregatorMerge":false,"descending":false,"granularity":{"type":"all"}}, peer=10.23.44.3} (java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.lang.RuntimeException: com.fasterxml.jackson.databind.exc.MismatchedInputException: Cannot construct instance of `org.apache.druid.segment.column.ColumnDescriptor` (although at least one Creator exists): no String-argument constructor/factory method to deserialize from String value ('valueType')
 at [Source: (StringReader); line: 1, column: 1])
2021-01-05T03:21:14,809 ERROR [qtp450476243-146[segmentMetadata_[traffic__ops_feed__realtime__second__dev__big__segment]_e94db4a3-21ed-42de-9471-6b1a83e74071]] org.apache.druid.server.QueryResource - Exception handling request: {class=org.apache.druid.server.QueryResource, exceptionType=class java.lang.RuntimeException, exceptionMessage=java.util.concurrent.ExecutionException: java.nio.BufferUnderflowException, query={"queryType":"segmentMetadata","dataSource":{"type":"table","name":"traffic__ops_feed__realtime__second__dev__big__segment"},"intervals":{"type":"segments","segments":[{"itvl":"2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z","ver":"2020-12-02T02:33:58.519Z","part":0},{"itvl":"2020-12-01T00:00:00.000Z/2020-12-02T00:00:00.000Z","ver":"2020-12-02T02:33:58.519Z","part":1}]},"toInclude":{"type":"all"},"merge":false,"context":{"defaultTimeout":60000,"finalize":false,"maxQueuedBytes":50000000,"maxScatterGatherBytes":9223372036854775807,"queryFailTime":1609816934586,"queryId":"e94db4a3-21ed-42de-9471-6b1a83e74071","timeout":60000},"analysisTypes":[],"usingDefaultInterval":false,"lenientAggregatorMerge":false,"descending":false,"granularity":{"type":"all"}}, peer=10.23.43.148} (java.lang.RuntimeException: java.util.concurrent.ExecutionException: java.nio.BufferUnderflowException)
2021-01-05T03:21:27,962 INFO [ZKCoordinator--3] org.apache.druid.server.coordination.SegmentLoadDropHandler - Loading segment traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z
2021-01-05T03:21:27,962 INFO [ZKCoordinator--3] org.apache.druid.storage.s3.S3DataSegmentPuller - Pulling index at path[CloudObjectLocation{bucket='pqm-druid-dev', path='rtstorage/segments/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0/index.zip'}] to outDir[/var/druid/segment-cache/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0]
2021-01-05T03:22:14,707 WARN [ZKCoordinator--3] com.amazonaws.services.s3.internal.S3AbortableInputStream - Not all bytes were read from the S3ObjectInputStream, aborting HTTP connection. This is likely an error and may result in sub-optimal behavior. Request only the bytes you need via a ranged GET or drain the input stream after use.
2021-01-05T03:22:14,707 WARN [ZKCoordinator--3] com.amazonaws.services.s3.internal.S3AbortableInputStream - Not all bytes were read from the S3ObjectInputStream, aborting HTTP connection. This is likely an error and may result in sub-optimal behavior. Request only the bytes you need via a ranged GET or drain the input stream after use.
2021-01-05T03:22:14,707 INFO [ZKCoordinator--3] org.apache.druid.storage.s3.S3DataSegmentPuller - Loaded 3388721170 bytes from [CloudObjectLocation{bucket='pqm-druid-dev', path='rtstorage/segments/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0/index.zip'}] to [/var/druid/segment-cache/traffic__ops_feed__realtime__second__dev__big__segment/2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z/2020-12-02T02:33:58.519Z/0]
2021-01-05T03:22:14,789 INFO [ZKCoordinator--3] org.apache.druid.server.coordination.BatchDataSegmentAnnouncer - Announcing segment[traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z] at existing path[/druid/segments/druid-dev-8-historical-0.druid-dev-8-historical.druid-dev-8.svc.cluster.local:8083/druid-dev-8-historical-0.druid-dev-8-historical.druid-dev-8.svc.cluster.local:8083_historical__default_tier_2021-01-05T03:09:36.189Z_bc91ea1098a749438ad4654b430c5cf61474]
2021-01-05T03:22:14,794 INFO [ZKCoordinator--3] org.apache.druid.server.coordination.ZkCoordinator - Completed request [LOAD: traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z]
2021-01-05T03:22:14,794 INFO [ZkCoordinator] org.apache.druid.server.coordination.ZkCoordinator - zNode[/druid/loadQueue/druid-dev-8-historical-0.druid-dev-8-historical.druid-dev-8.svc.cluster.local:8083/traffic__ops_feed__realtime__second__dev__big__segment_2020-12-01T00:00:00.000Z_2020-12-02T00:00:00.000Z_2020-12-02T02:33:58.519Z] was removed

damage segment files -> lazy On start -> query failed -> segment unload automatically -> load segment according to coordinator -> query successfully.

@kaijianding kaijianding self-assigned this Jan 6, 2021
Copy link
Copy Markdown
Contributor

@kaijianding kaijianding left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overall, LGTM except for the naming.

And add a unit test to verify the callback called when failed to deserialize column, and meet the coverage requirement of the CI


package org.apache.druid.coordination;

public interface CommonCallback
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Use a more meaningful name, like SegmentLazyLoadFailCallback

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.

{
return loadIndex(inDir, false);
return loadIndex(inDir, false, CommonCallback.NOOP);
}
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add blank line here

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.


private ColumnHolder deserializeColumn(ObjectMapper mapper, ByteBuffer byteBuffer, SmooshedFileMapper smooshedFiles)
throws IOException
throws IOException, RuntimeException
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why RuntimeException here?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need actually. Remove now.

* under the License.
*/

package org.apache.druid.coordination;
Copy link
Copy Markdown
Contributor

@kaijianding kaijianding Jan 6, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I guess this interface should be in org.apache.druid.segment, beside the IndexIO

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done.


package org.apache.druid.coordination;

public interface CommonCallback
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why not just use Runnable? There is even a Runnables.getNoopRunnable()

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think a meaningful name is better for code readiness through the method arguments

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't feel super strongly about this, while I find it pretty obvious what Runnable does and don't think a new interface adds much here, I can see your point as well.

I would agree that if we insist on using a separate interface that it should have a better name more indicative of its function.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Change the name to SegmentLazyLoadFailCallback as you guys suggested. Thanks for review :)

* @throws SegmentLoadingException if the segment cannot be loaded
*/
public boolean loadSegment(final DataSegment segment, boolean lazy) throws SegmentLoadingException
public boolean loadSegment(final DataSegment segment, boolean lazy, CommonCallback loadFailed) throws SegmentLoadingException
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

add a new @param in the method comment

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

@kaijianding
Copy link
Copy Markdown
Contributor

And rename the PR title to match the changes

@jihoonson jihoonson added the Bug label Jan 13, 2021
Comment thread .travis.yml
# resolving the SIGAR dependency.
- >
MAVEN_OPTS='-Xmx800m' ${MVN} test -pl ${MAVEN_PROJECTS}
MAVEN_OPTS='-Xmx1100m' ${MVN} test -pl ${MAVEN_PROJECTS}
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what is this change for?

Copy link
Copy Markdown
Contributor Author

@zhangyue19921010 zhangyue19921010 Jan 15, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Current UTs don't cover load segment in a lazy way. This PR add a UT to test lazy loading which needs extra memory.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍 was just surprised how much more it needed, if travis passes then I assume the change is fine, was just curious mostly

@clintropolis
Copy link
Copy Markdown
Member

Hi @zhangyue19921010

I know some of our travis integration tests have been a bit flakey lately, but could you avoid doing this to trigger CI:
Screen Shot 2021-01-15 at 12 23 25 AM
It just slows things down for all PRs. A committer will usually restart any jobs which appear to have been an intermittent failure when doing a review.

@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

Hi @zhangyue19921010

I know some of our travis integration tests have been a bit flakey lately, but could you avoid doing this to trigger CI:
Screen Shot 2021-01-15 at 12 23 25 AM
It just slows things down for all PRs. A committer will usually restart any jobs which appear to have been an intermittent failure when doing a review.

Sure, Sorry for that. I don't have the permission for re-running a single job so I have to re-run the whole CI :(
Express apologies once again.
I will wait for reviewers to re-run the failed job.

@clintropolis clintropolis changed the title Historical reloads damaged segments automatically when lazy on start. Historical unloads damaged segments automatically when lazy on start. Jan 17, 2021
Copy link
Copy Markdown
Member

@clintropolis clintropolis left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

lgtm, thanks 👍

@clintropolis clintropolis merged commit 2590ad4 into apache:master Jan 17, 2021
@zhangyue19921010
Copy link
Copy Markdown
Contributor Author

@kaijianding @clintropolis Thanks for your review and merge!

JulianJaffePinterest pushed a commit to JulianJaffePinterest/druid that referenced this pull request Jan 22, 2021
…apache#10688)

* ready to test

* tested on dev cluster

* tested

* code review

* add UTs

* add UTs

* ut passed

* ut passed

* opti imports

* done

* done

* fix checkstyle

* modify uts

* modify logs

* changing the package of SegmentLazyLoadFailCallback.java to org.apache.druid.segment

* merge from master

* modify import orders

* merge from master

* merge from master

* modify logs

* modify docs

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

* modify logs to rerun ci

Co-authored-by: yuezhang <yuezhang@freewheel.tv>
@zhangyue19921010 zhangyue19921010 deleted the historical-lazyOnStart-with-fileCheck branch February 9, 2021 08:42
@clintropolis clintropolis added this to the 0.22.0 milestone Aug 12, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants