-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Fix MSQ compaction state and native interval locking, add test coverage #18950
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
Changes from all commits
dd3293a
f723153
4004d08
2531140
a335f9d
02f8030
9050d70
e43ec58
80ead3d
c3de9ea
5d17e68
95f4e09
f918259
58c7197
4024d87
d56410f
c79ca10
05c8688
8e44e7c
529dcd7
c99ee19
27fa2ed
f4af751
c19202e
d85d0fc
1fb1342
c38fbe1
7221970
782d2ac
241b2a7
1079703
3c7df1a
a0ed966
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,6 +20,7 @@ | |
| package org.apache.druid.indexing.common.actions; | ||
|
|
||
| import com.fasterxml.jackson.databind.ObjectMapper; | ||
| import com.google.common.base.Preconditions; | ||
| import com.google.common.base.Suppliers; | ||
| import org.apache.druid.indexing.common.TestUtils; | ||
| import org.apache.druid.indexing.common.config.TaskStorageConfig; | ||
|
|
@@ -48,10 +49,11 @@ | |
| import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; | ||
| import org.apache.druid.server.coordinator.simulate.TestDruidLeaderSelector; | ||
| import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; | ||
| import org.easymock.EasyMock; | ||
| import org.joda.time.Period; | ||
| import org.junit.rules.ExternalResource; | ||
|
|
||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
|
|
||
| public class TaskActionTestKit extends ExternalResource | ||
| { | ||
| private final MetadataStorageTablesConfig metadataStorageTablesConfig = MetadataStorageTablesConfig.fromBase("druid"); | ||
|
|
@@ -66,7 +68,46 @@ public class TaskActionTestKit extends ExternalResource | |
| private BlockingExecutorService metadataCachePollExec; | ||
|
|
||
| private boolean useSegmentMetadataCache = false; | ||
| private boolean useCentralizedDatasourceSchema = false; | ||
| private boolean batchSegmentAllocation = true; | ||
| private boolean skipSegmentPayloadFetchForAllocation = new TaskLockConfig().isBatchAllocationReduceMetadataIO(); | ||
| private AtomicBoolean configFinalized = new AtomicBoolean(); | ||
|
|
||
| public TaskActionTestKit setUseSegmentMetadataCache(boolean useSegmentMetadataCache) | ||
| { | ||
| if (configFinalized.get()) { | ||
| throw new IllegalStateException("Test config already finalized"); | ||
| } | ||
| this.useSegmentMetadataCache = useSegmentMetadataCache; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskActionTestKit setUseCentralizedDatasourceSchema(boolean useCentralizedDatasourceSchema) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Did centralized schema also run into an issue similar to batch allocation?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it's not supported by msq engine, schema is not saved |
||
| { | ||
| if (configFinalized.get()) { | ||
| throw new IllegalStateException("Test config already finalized"); | ||
| } | ||
| this.useCentralizedDatasourceSchema = useCentralizedDatasourceSchema; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskActionTestKit setBatchSegmentAllocation(boolean batchSegmentAllocation) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just curious, have we added new tests which need this?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ah i think i had some issues with batch segment allocation with segment lock, and some pending segments cache was not removed properly or something (maybe i should have written it down), and have decided to disable the batch allocation to avoid this issue
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yea so it's set to false in CompactionTaskRunBase
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, if that is the case, we would need to dig deeper into it since it sounds like a bug and batch segment allocation is the default. It should work for all cases.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it's only for segment lock, and i wasn't sure it's due to test setup or anything.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Okay, in that case, let's skip the test case for segment lock and add a comment or add a test case and mark it disabled. That way, we would be aware that there is a bug and can address it later.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i could not reproduce the test cases any more, maybe it disappeared after we switched to use the widen interval. anyways, so i added the segmentQueue test param.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Oh, okay. So do the tests now use batch segment allocation or not? |
||
| { | ||
| if (configFinalized.get()) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we want to impose this limit? It should be okay to be able to change the config between tests.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the before method was called before @test, wont that mean we can't change the config between tests, but in setup only?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. well this is just to prevent ppl from accidently calling it inside test method
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why do we need to protect devs against that? This is test-only code.
If the setup method (i.e. |
||
| throw new IllegalStateException("Test config already finalized"); | ||
| } | ||
| this.batchSegmentAllocation = batchSegmentAllocation; | ||
| return this; | ||
| } | ||
|
|
||
| public TaskActionTestKit setSkipSegmentPayloadFetchForAllocation(boolean skipSegmentPayloadFetchForAllocation) | ||
| { | ||
| if (configFinalized.get()) { | ||
| throw new IllegalStateException("Test config already finalized"); | ||
| } | ||
| this.skipSegmentPayloadFetchForAllocation = skipSegmentPayloadFetchForAllocation; | ||
| return this; | ||
| } | ||
|
|
||
| public StubServiceEmitter getServiceEmitter() | ||
| { | ||
|
|
@@ -88,6 +129,11 @@ public GlobalTaskLockbox getTaskLockbox() | |
| return taskLockbox; | ||
| } | ||
|
|
||
| public TaskStorage getTaskStorage() | ||
| { | ||
| return taskStorage; | ||
| } | ||
|
|
||
| public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator() | ||
| { | ||
| return metadataStorageCoordinator; | ||
|
|
@@ -98,16 +144,6 @@ public TaskActionToolbox getTaskActionToolbox() | |
| return taskActionToolbox; | ||
| } | ||
|
|
||
| public void setSkipSegmentPayloadFetchForAllocation(boolean skipSegmentPayloadFetchForAllocation) | ||
| { | ||
| this.skipSegmentPayloadFetchForAllocation = skipSegmentPayloadFetchForAllocation; | ||
| } | ||
|
|
||
| public void setUseSegmentMetadataCache(boolean useSegmentMetadataCache) | ||
| { | ||
| this.useSegmentMetadataCache = useSegmentMetadataCache; | ||
| } | ||
|
|
||
| public void syncSegmentMetadataCache() | ||
| { | ||
| metadataCachePollExec.finishNextPendingTasks(4); | ||
|
|
@@ -116,11 +152,13 @@ public void syncSegmentMetadataCache() | |
| @Override | ||
| public void before() | ||
| { | ||
| Preconditions.checkState(configFinalized.compareAndSet(false, true)); | ||
| emitter = new StubServiceEmitter(); | ||
| taskStorage = new HeapMemoryTaskStorage(new TaskStorageConfig(new Period("PT24H"))); | ||
| testDerbyConnector = new TestDerbyConnector( | ||
| new MetadataStorageConnectorConfig(), | ||
| metadataStorageTablesConfig | ||
| metadataStorageTablesConfig, | ||
| CentralizedDatasourceSchemaConfig.enabled(useCentralizedDatasourceSchema) | ||
| ); | ||
| final ObjectMapper objectMapper = new TestUtils().getTestObjectMapper(); | ||
| final SegmentSchemaManager segmentSchemaManager = new SegmentSchemaManager( | ||
|
|
@@ -136,13 +174,19 @@ public void before() | |
| metadataStorageTablesConfig, | ||
| testDerbyConnector, | ||
| segmentSchemaManager, | ||
| CentralizedDatasourceSchemaConfig.create(), | ||
| CentralizedDatasourceSchemaConfig.enabled(useCentralizedDatasourceSchema), | ||
| new HeapMemoryIndexingStateStorage() | ||
| ); | ||
| taskLockbox = new GlobalTaskLockbox(taskStorage, metadataStorageCoordinator); | ||
| taskLockbox.syncFromStorage(); | ||
| final TaskLockConfig taskLockConfig = new TaskLockConfig() | ||
| { | ||
| @Override | ||
| public boolean isBatchSegmentAllocation() | ||
| { | ||
| return batchSegmentAllocation; | ||
| } | ||
|
|
||
| @Override | ||
| public long getBatchAllocationWaitTime() | ||
| { | ||
|
|
@@ -156,22 +200,28 @@ public boolean isBatchAllocationReduceMetadataIO() | |
| } | ||
| }; | ||
|
|
||
| SupervisorManager supervisorManager = new SupervisorManager(objectMapper, null); | ||
| SegmentAllocationQueue segmentAllocationQueue = new SegmentAllocationQueue( | ||
| taskLockbox, | ||
| taskLockConfig, | ||
| metadataStorageCoordinator, | ||
| emitter, | ||
| ScheduledExecutors::fixed | ||
| ); | ||
| if (segmentAllocationQueue.isEnabled()) { | ||
| segmentAllocationQueue.becomeLeader(); | ||
| } | ||
| taskActionToolbox = new TaskActionToolbox( | ||
| taskLockbox, | ||
| taskStorage, | ||
| metadataStorageCoordinator, | ||
| new SegmentAllocationQueue( | ||
| taskLockbox, | ||
| taskLockConfig, | ||
| metadataStorageCoordinator, | ||
| emitter, | ||
| ScheduledExecutors::fixed | ||
| ), | ||
| segmentAllocationQueue, | ||
| emitter, | ||
| EasyMock.createMock(SupervisorManager.class), | ||
| supervisorManager, | ||
| objectMapper | ||
| ); | ||
| testDerbyConnector.createDataSourceTable(); | ||
| testDerbyConnector.createUpgradeSegmentsTable(); | ||
| testDerbyConnector.createPendingSegmentsTable(); | ||
| testDerbyConnector.createSegmentSchemasTable(); | ||
| testDerbyConnector.createSegmentTable(); | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.