mirror of https://github.com/apache/druid.git
Clean up SqlSegmentsMetadataManager and corresponding tests (#16044)
Changes: Improve `SqlSegmentsMetadataManager` - Break the loop in `populateUsedStatusLastUpdated` before going to sleep if there are no more segments to update - Add comments and clean up logs Refactor `SqlSegmentsMetadataManagerTest` - Merge `SqlSegmentsMetadataManagerEmptyTest` into this test - Add method `testPollEmpty` - Shave a few seconds off of the tests by reducing poll duration - Simplify creation of test segments - Some renames here and there - Remove unused methods - Move `TestDerbyConnector.allowLastUsedFlagToBeNull` to this class Other minor changes - Add javadoc to `NoneShardSpec` - Use lambda in `SqlSegmentMetadataPublisher`
This commit is contained in:
parent
3caacba8c5
commit
5f203725dd
|
@ -29,6 +29,16 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* {@link ShardSpec} with no partitioning in a time chunk, i.e. a single segment
|
||||
* per time chunk. This shard spec has been deprecated and is not generated by
|
||||
* the Druid code anymore. The class has been retained only for backward
|
||||
* compatibility reasons.
|
||||
* <p>
|
||||
* For more information, refer to
|
||||
* <a href="https://github.com/apache/druid/pull/6883">PR #6883</a>.
|
||||
*
|
||||
* @deprecated Since Druid 0.15.0. Segments generated by Druid 0.15.0 onwards
|
||||
* do not use this shard spec.
|
||||
*/
|
||||
@Deprecated
|
||||
public class NoneShardSpec implements ShardSpec
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.apache.druid.metadata;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.DateTimes;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
|
@ -28,8 +27,6 @@ import org.apache.druid.java.util.common.logger.Logger;
|
|||
import org.apache.druid.timeline.DataSegment;
|
||||
import org.apache.druid.timeline.partition.NoneShardSpec;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
@ -79,8 +76,7 @@ public class SQLMetadataSegmentPublisher implements MetadataSegmentPublisher
|
|||
);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
void publishSegment(
|
||||
private void publishSegment(
|
||||
final String segmentId,
|
||||
final String dataSource,
|
||||
final String createdDate,
|
||||
|
@ -96,31 +92,18 @@ public class SQLMetadataSegmentPublisher implements MetadataSegmentPublisher
|
|||
try {
|
||||
final DBI dbi = connector.getDBI();
|
||||
List<Map<String, Object>> exists = dbi.withHandle(
|
||||
new HandleCallback<List<Map<String, Object>>>()
|
||||
{
|
||||
@Override
|
||||
public List<Map<String, Object>> withHandle(Handle handle)
|
||||
{
|
||||
return handle.createQuery(
|
||||
StringUtils.format("SELECT id FROM %s WHERE id=:id", config.getSegmentsTable())
|
||||
)
|
||||
.bind("id", segmentId)
|
||||
.list();
|
||||
}
|
||||
}
|
||||
handle -> handle.createQuery(
|
||||
StringUtils.format("SELECT id FROM %s WHERE id=:id", config.getSegmentsTable())
|
||||
).bind("id", segmentId).list()
|
||||
);
|
||||
|
||||
if (!exists.isEmpty()) {
|
||||
log.info("Found [%s] in DB, not updating DB", segmentId);
|
||||
log.info("Skipping publish of segment[%s] as it already exists in the metadata store.", segmentId);
|
||||
return;
|
||||
}
|
||||
|
||||
dbi.withHandle(
|
||||
new HandleCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle)
|
||||
{
|
||||
handle ->
|
||||
handle.createStatement(statement)
|
||||
.bind("id", segmentId)
|
||||
.bind("dataSource", dataSource)
|
||||
|
@ -132,11 +115,7 @@ public class SQLMetadataSegmentPublisher implements MetadataSegmentPublisher
|
|||
.bind("used", used)
|
||||
.bind("payload", payload)
|
||||
.bind("used_status_last_updated", usedFlagLastUpdated)
|
||||
.execute();
|
||||
|
||||
return null;
|
||||
}
|
||||
}
|
||||
.execute()
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
|
|
@ -64,7 +64,6 @@ import org.skife.jdbi.v2.Query;
|
|||
import org.skife.jdbi.v2.StatementContext;
|
||||
import org.skife.jdbi.v2.TransactionCallback;
|
||||
import org.skife.jdbi.v2.TransactionStatus;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
import org.skife.jdbi.v2.tweak.ResultSetMapper;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
@ -72,6 +71,7 @@ import java.io.IOException;
|
|||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
|
@ -335,7 +335,7 @@ public class SqlSegmentsMetadataManager implements SegmentsMetadataManager
|
|||
{
|
||||
ExecutorService executorService = Executors.newSingleThreadExecutor();
|
||||
usedFlagLastUpdatedPopulationFuture = executorService.submit(
|
||||
() -> populateUsedFlagLastUpdated()
|
||||
this::populateUsedFlagLastUpdated
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -347,70 +347,68 @@ public class SqlSegmentsMetadataManager implements SegmentsMetadataManager
|
|||
@VisibleForTesting
|
||||
void populateUsedFlagLastUpdated()
|
||||
{
|
||||
String segmentsTable = getSegmentsTable();
|
||||
final String segmentsTable = getSegmentsTable();
|
||||
log.info(
|
||||
"Populating used_status_last_updated with non-NULL values for unused segments in [%s]",
|
||||
"Populating column 'used_status_last_updated' with non-NULL values for unused segments in table[%s].",
|
||||
segmentsTable
|
||||
);
|
||||
|
||||
int limit = 100;
|
||||
final int batchSize = 100;
|
||||
int totalUpdatedEntries = 0;
|
||||
|
||||
// Update the rows in batches of size 100
|
||||
while (true) {
|
||||
List<String> segmentsToUpdate = new ArrayList<>(100);
|
||||
final List<String> segmentsToUpdate = new ArrayList<>(batchSize);
|
||||
int numUpdatedRows;
|
||||
try {
|
||||
connector.retryWithHandle(
|
||||
new HandleCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle)
|
||||
{
|
||||
segmentsToUpdate.addAll(handle.createQuery(
|
||||
StringUtils.format(
|
||||
"SELECT id FROM %1$s WHERE used_status_last_updated IS NULL and used = :used %2$s",
|
||||
segmentsTable,
|
||||
connector.limitClause(limit)
|
||||
)
|
||||
).bind("used", false).mapTo(String.class).list());
|
||||
return null;
|
||||
}
|
||||
handle -> {
|
||||
segmentsToUpdate.addAll(handle.createQuery(
|
||||
StringUtils.format(
|
||||
"SELECT id FROM %1$s WHERE used_status_last_updated IS NULL and used = :used %2$s",
|
||||
segmentsTable,
|
||||
connector.limitClause(batchSize)
|
||||
)
|
||||
).bind("used", false).mapTo(String.class).list());
|
||||
return null;
|
||||
}
|
||||
);
|
||||
|
||||
if (segmentsToUpdate.isEmpty()) {
|
||||
// We have no segments to process
|
||||
break;
|
||||
}
|
||||
|
||||
connector.retryWithHandle(
|
||||
new HandleCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle)
|
||||
{
|
||||
Batch updateBatch = handle.createBatch();
|
||||
String sql = "UPDATE %1$s SET used_status_last_updated = '%2$s' WHERE id = '%3$s'";
|
||||
String now = DateTimes.nowUtc().toString();
|
||||
for (String id : segmentsToUpdate) {
|
||||
updateBatch.add(StringUtils.format(sql, segmentsTable, now, id));
|
||||
}
|
||||
updateBatch.execute();
|
||||
return null;
|
||||
numUpdatedRows = connector.retryWithHandle(
|
||||
handle -> {
|
||||
final Batch updateBatch = handle.createBatch();
|
||||
final String sql = "UPDATE %1$s SET used_status_last_updated = '%2$s' WHERE id = '%3$s'";
|
||||
String now = DateTimes.nowUtc().toString();
|
||||
for (String id : segmentsToUpdate) {
|
||||
updateBatch.add(StringUtils.format(sql, segmentsTable, now, id));
|
||||
}
|
||||
int[] results = updateBatch.execute();
|
||||
return Arrays.stream(results).sum();
|
||||
}
|
||||
);
|
||||
totalUpdatedEntries += numUpdatedRows;
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.warn(e, "Population of used_status_last_updated in [%s] has failed. There may be unused segments with"
|
||||
+ " NULL values for used_status_last_updated that won't be killed!", segmentsTable);
|
||||
log.warn(e, "Populating column 'used_status_last_updated' in table[%s] has failed. There may be unused segments with"
|
||||
+ " NULL values for 'used_status_last_updated' that won't be killed!", segmentsTable);
|
||||
return;
|
||||
}
|
||||
|
||||
totalUpdatedEntries += segmentsToUpdate.size();
|
||||
log.info("Updated a batch of %d rows in [%s] with a valid used_status_last_updated date",
|
||||
segmentsToUpdate.size(),
|
||||
segmentsTable
|
||||
log.debug(
|
||||
"Updated a batch of [%d] rows in table[%s] with a valid used_status_last_updated date",
|
||||
segmentsToUpdate.size(), segmentsTable
|
||||
);
|
||||
|
||||
// Do not wait if there are no more segments to update
|
||||
if (segmentsToUpdate.size() == numUpdatedRows && numUpdatedRows < batchSize) {
|
||||
break;
|
||||
}
|
||||
|
||||
// Wait for some time before processing the next batch
|
||||
try {
|
||||
Thread.sleep(10000);
|
||||
}
|
||||
|
@ -420,9 +418,8 @@ public class SqlSegmentsMetadataManager implements SegmentsMetadataManager
|
|||
}
|
||||
}
|
||||
log.info(
|
||||
"Finished updating [%s] with a valid used_status_last_updated date. %d rows updated",
|
||||
segmentsTable,
|
||||
totalUpdatedEntries
|
||||
"Populated column 'used_status_last_updated' in table[%s] in [%d] rows.",
|
||||
segmentsTable, totalUpdatedEntries
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -1,113 +0,0 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.metadata;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import org.apache.druid.client.ImmutableDruidDataSource;
|
||||
import org.apache.druid.segment.TestHelper;
|
||||
import org.joda.time.Period;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
||||
/**
|
||||
* Like {@link SQLMetadataRuleManagerTest} except with no segments to make sure it behaves when it's empty
|
||||
*/
|
||||
public class SqlSegmentsMetadataManagerEmptyTest
|
||||
{
|
||||
|
||||
@Rule
|
||||
public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule();
|
||||
|
||||
private SqlSegmentsMetadataManager sqlSegmentsMetadataManager;
|
||||
private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
|
||||
|
||||
@Before
|
||||
public void setUp()
|
||||
{
|
||||
TestDerbyConnector connector = derbyConnectorRule.getConnector();
|
||||
SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig();
|
||||
config.setPollDuration(Period.seconds(1));
|
||||
sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager(
|
||||
jsonMapper,
|
||||
Suppliers.ofInstance(config),
|
||||
derbyConnectorRule.metadataTablesConfigSupplier(),
|
||||
connector
|
||||
);
|
||||
sqlSegmentsMetadataManager.start();
|
||||
|
||||
connector.createSegmentTable();
|
||||
}
|
||||
|
||||
@After
|
||||
public void teardown()
|
||||
{
|
||||
if (sqlSegmentsMetadataManager.isPollingDatabasePeriodically()) {
|
||||
sqlSegmentsMetadataManager.stopPollingDatabasePeriodically();
|
||||
}
|
||||
sqlSegmentsMetadataManager.stop();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPollEmpty()
|
||||
{
|
||||
sqlSegmentsMetadataManager.startPollingDatabasePeriodically();
|
||||
sqlSegmentsMetadataManager.poll();
|
||||
Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically());
|
||||
Assert.assertEquals(
|
||||
ImmutableSet.of(),
|
||||
sqlSegmentsMetadataManager.retrieveAllDataSourceNames()
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
sqlSegmentsMetadataManager
|
||||
.getImmutableDataSourcesWithAllUsedSegments()
|
||||
.stream()
|
||||
.map(ImmutableDruidDataSource::getName)
|
||||
.collect(Collectors.toList())
|
||||
);
|
||||
Assert.assertEquals(
|
||||
null,
|
||||
sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments("wikipedia")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableSet.of(),
|
||||
ImmutableSet.copyOf(sqlSegmentsMetadataManager.iterateAllUsedSegments())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStopAndStart()
|
||||
{
|
||||
// Simulate successive losing and getting the coordinator leadership
|
||||
sqlSegmentsMetadataManager.startPollingDatabasePeriodically();
|
||||
sqlSegmentsMetadataManager.stopPollingDatabasePeriodically();
|
||||
sqlSegmentsMetadataManager.startPollingDatabasePeriodically();
|
||||
sqlSegmentsMetadataManager.stopPollingDatabasePeriodically();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -25,14 +25,10 @@ import org.apache.druid.java.util.common.StringUtils;
|
|||
import org.apache.druid.metadata.storage.derby.DerbyConnector;
|
||||
import org.junit.Assert;
|
||||
import org.junit.rules.ExternalResource;
|
||||
import org.skife.jdbi.v2.Batch;
|
||||
import org.skife.jdbi.v2.DBI;
|
||||
import org.skife.jdbi.v2.Handle;
|
||||
import org.skife.jdbi.v2.exceptions.UnableToObtainConnectionException;
|
||||
import org.skife.jdbi.v2.tweak.HandleCallback;
|
||||
|
||||
import java.sql.SQLException;
|
||||
import java.util.Locale;
|
||||
import java.util.UUID;
|
||||
|
||||
public class TestDerbyConnector extends DerbyConnector
|
||||
|
@ -139,27 +135,5 @@ public class TestDerbyConnector extends DerbyConnector
|
|||
{
|
||||
return dbTables;
|
||||
}
|
||||
|
||||
public void allowUsedFlagLastUpdatedToBeNullable()
|
||||
{
|
||||
connector.retryWithHandle(
|
||||
new HandleCallback<Void>()
|
||||
{
|
||||
@Override
|
||||
public Void withHandle(Handle handle)
|
||||
{
|
||||
final Batch batch = handle.createBatch();
|
||||
batch.add(
|
||||
StringUtils.format(
|
||||
"ALTER TABLE %1$s ALTER COLUMN USED_STATUS_LAST_UPDATED NULL",
|
||||
dbTables.get().getSegmentsTable().toUpperCase(Locale.ENGLISH)
|
||||
)
|
||||
);
|
||||
batch.execute();
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue