mirror of https://github.com/apache/druid.git
maxTotalRows should be checked in DataSourceCompactionConfig before setting targetCompactionSizeBytes (#7368)
* maxTotalRows should be checked in DataSourceCompactionConfig before setting targetCompactionSizeBytes * remove unnecessary default values * remove flacky test * fix build * Add comments
This commit is contained in:
parent
a09aa13ead
commit
62c3e89266
|
@ -64,7 +64,7 @@ public class ITHadoopIndexTest extends AbstractIndexerTest
|
||||||
|
|
||||||
try {
|
try {
|
||||||
LOG.info("indexerFile name: [%s]", BATCH_TASK);
|
LOG.info("indexerFile name: [%s]", BATCH_TASK);
|
||||||
indexerSpec = getTaskAsString(BATCH_TASK);
|
indexerSpec = getResourceAsString(BATCH_TASK);
|
||||||
indexerSpec = StringUtils.replace(indexerSpec, "%%HADOOP_TEST_PATH%%", hadoopDir);
|
indexerSpec = StringUtils.replace(indexerSpec, "%%HADOOP_TEST_PATH%%", hadoopDir);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -56,7 +56,7 @@ public class AbstractITBatchIndexTest extends AbstractIndexerTest
|
||||||
{
|
{
|
||||||
final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
|
final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
|
||||||
final String taskSpec = StringUtils.replace(
|
final String taskSpec = StringUtils.replace(
|
||||||
getTaskAsString(indexTaskFilePath),
|
getResourceAsString(indexTaskFilePath),
|
||||||
"%%DATASOURCE%%",
|
"%%DATASOURCE%%",
|
||||||
fullDatasourceName
|
fullDatasourceName
|
||||||
);
|
);
|
||||||
|
@ -98,7 +98,7 @@ public class AbstractITBatchIndexTest extends AbstractIndexerTest
|
||||||
final String fullReindexDatasourceName = reindexDataSource + config.getExtraDatasourceNameSuffix();
|
final String fullReindexDatasourceName = reindexDataSource + config.getExtraDatasourceNameSuffix();
|
||||||
|
|
||||||
String taskSpec = StringUtils.replace(
|
String taskSpec = StringUtils.replace(
|
||||||
getTaskAsString(reindexTaskFilePath),
|
getResourceAsString(reindexTaskFilePath),
|
||||||
"%%DATASOURCE%%",
|
"%%DATASOURCE%%",
|
||||||
fullBaseDatasourceName
|
fullBaseDatasourceName
|
||||||
);
|
);
|
||||||
|
@ -148,7 +148,7 @@ public class AbstractITBatchIndexTest extends AbstractIndexerTest
|
||||||
{
|
{
|
||||||
final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
|
final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();
|
||||||
final String taskSpec = StringUtils.replace(
|
final String taskSpec = StringUtils.replace(
|
||||||
getTaskAsString(indexTaskFilePath),
|
getResourceAsString(indexTaskFilePath),
|
||||||
"%%DATASOURCE%%",
|
"%%DATASOURCE%%",
|
||||||
fullDatasourceName
|
fullDatasourceName
|
||||||
);
|
);
|
||||||
|
|
|
@ -88,7 +88,7 @@ public abstract class AbstractITRealtimeIndexTaskTest extends AbstractIndexerTes
|
||||||
try (final Closeable closeable = unloader(fullDatasourceName)) {
|
try (final Closeable closeable = unloader(fullDatasourceName)) {
|
||||||
// the task will run for 3 minutes and then shutdown itself
|
// the task will run for 3 minutes and then shutdown itself
|
||||||
String task = setShutOffTime(
|
String task = setShutOffTime(
|
||||||
getTaskAsString(getTaskResource()),
|
getResourceAsString(getTaskResource()),
|
||||||
DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3))
|
DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3))
|
||||||
);
|
);
|
||||||
task = StringUtils.replace(task, "%%DATASOURCE%%", fullDatasourceName);
|
task = StringUtils.replace(task, "%%DATASOURCE%%", fullDatasourceName);
|
||||||
|
|
|
@ -112,7 +112,7 @@ public abstract class AbstractIndexerTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
protected String getTaskAsString(String file) throws IOException
|
protected String getResourceAsString(String file) throws IOException
|
||||||
{
|
{
|
||||||
final InputStream inputStream = ITRealtimeIndexTaskTest.class.getResourceAsStream(file);
|
final InputStream inputStream = ITRealtimeIndexTaskTest.class.getResourceAsStream(file);
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -138,7 +138,7 @@ public class AbstractKafkaIndexerTest extends AbstractIndexerTest
|
||||||
consumerProperties.putAll(consumerConfigs);
|
consumerProperties.putAll(consumerConfigs);
|
||||||
consumerProperties.put("bootstrap.servers", config.getKafkaInternalHost());
|
consumerProperties.put("bootstrap.servers", config.getKafkaInternalHost());
|
||||||
|
|
||||||
spec = getTaskAsString(INDEXER_FILE);
|
spec = getResourceAsString(INDEXER_FILE);
|
||||||
spec = StringUtils.replace(spec, "%%DATASOURCE%%", fullDatasourceName);
|
spec = StringUtils.replace(spec, "%%DATASOURCE%%", fullDatasourceName);
|
||||||
spec = StringUtils.replace(spec, "%%TOPIC%%", TOPIC_NAME);
|
spec = StringUtils.replace(spec, "%%TOPIC%%", TOPIC_NAME);
|
||||||
spec = StringUtils.replace(spec, "%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties));
|
spec = StringUtils.replace(spec, "%%CONSUMER_PROPERTIES%%", jsonMapper.writeValueAsString(consumerProperties));
|
||||||
|
|
|
@ -131,7 +131,7 @@ public class ITCompactionTaskTest extends AbstractIndexerTest
|
||||||
|
|
||||||
private void loadData() throws Exception
|
private void loadData() throws Exception
|
||||||
{
|
{
|
||||||
String taskSpec = getTaskAsString(INDEX_TASK);
|
String taskSpec = getResourceAsString(INDEX_TASK);
|
||||||
taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
|
taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
|
||||||
final String taskID = indexer.submitTask(taskSpec);
|
final String taskID = indexer.submitTask(taskSpec);
|
||||||
LOG.info("TaskID for loading index task %s", taskID);
|
LOG.info("TaskID for loading index task %s", taskID);
|
||||||
|
@ -145,7 +145,7 @@ public class ITCompactionTaskTest extends AbstractIndexerTest
|
||||||
|
|
||||||
private void compactData(boolean keepSegmentGranularity) throws Exception
|
private void compactData(boolean keepSegmentGranularity) throws Exception
|
||||||
{
|
{
|
||||||
final String template = getTaskAsString(COMPACTION_TASK);
|
final String template = getResourceAsString(COMPACTION_TASK);
|
||||||
String taskSpec =
|
String taskSpec =
|
||||||
StringUtils.replace(template, "${KEEP_SEGMENT_GRANULARITY}", Boolean.toString(keepSegmentGranularity));
|
StringUtils.replace(template, "${KEEP_SEGMENT_GRANULARITY}", Boolean.toString(keepSegmentGranularity));
|
||||||
taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
|
taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
|
||||||
|
|
|
@ -96,7 +96,7 @@ public class ITNestedQueryPushDownTest extends AbstractIndexerTest
|
||||||
|
|
||||||
private void loadData() throws Exception
|
private void loadData() throws Exception
|
||||||
{
|
{
|
||||||
String taskSpec = getTaskAsString(WIKITICKER_INDEX_TASK);
|
String taskSpec = getResourceAsString(WIKITICKER_INDEX_TASK);
|
||||||
taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
|
taskSpec = StringUtils.replace(taskSpec, "%%DATASOURCE%%", fullDatasourceName);
|
||||||
final String taskID = indexer.submitTask(taskSpec);
|
final String taskID = indexer.submitTask(taskSpec);
|
||||||
LOG.info("TaskID for loading index task %s", taskID);
|
LOG.info("TaskID for loading index task %s", taskID);
|
||||||
|
|
|
@ -93,7 +93,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest
|
||||||
try {
|
try {
|
||||||
// Load 4 datasources with same dimensions
|
// Load 4 datasources with same dimensions
|
||||||
String task = setShutOffTime(
|
String task = setShutOffTime(
|
||||||
getTaskAsString(UNION_TASK_RESOURCE),
|
getResourceAsString(UNION_TASK_RESOURCE),
|
||||||
DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3))
|
DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3))
|
||||||
);
|
);
|
||||||
List<String> taskIDs = new ArrayList<>();
|
List<String> taskIDs = new ArrayList<>();
|
||||||
|
|
|
@ -99,4 +99,17 @@ public class ClientCompactQuery implements ClientQuery
|
||||||
{
|
{
|
||||||
return context;
|
return context;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString()
|
||||||
|
{
|
||||||
|
return "ClientCompactQuery{" +
|
||||||
|
"dataSource='" + dataSource + '\'' +
|
||||||
|
", segments=" + segments +
|
||||||
|
", keepSegmentGranularity=" + keepSegmentGranularity +
|
||||||
|
", targetCompactionSizeBytes=" + targetCompactionSizeBytes +
|
||||||
|
", tuningConfig=" + tuningConfig +
|
||||||
|
", context=" + context +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -29,20 +29,18 @@ import java.util.Objects;
|
||||||
|
|
||||||
public class ClientCompactQueryTuningConfig
|
public class ClientCompactQueryTuningConfig
|
||||||
{
|
{
|
||||||
// These default values should be synchronized with those of IndexTuningConfig
|
|
||||||
private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000;
|
|
||||||
private static final int DEFAULT_MAX_TOTAL_ROWS = 20_000_000;
|
|
||||||
private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec();
|
|
||||||
private static final int DEFAULT_MAX_PENDING_PERSISTS = 0;
|
|
||||||
private static final long DEFAULT_PUSH_TIMEOUT = 0;
|
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
private final Integer maxRowsPerSegment;
|
private final Integer maxRowsPerSegment;
|
||||||
private final int maxRowsInMemory;
|
@Nullable
|
||||||
private final int maxTotalRows;
|
private final Integer maxRowsInMemory;
|
||||||
|
@Nullable
|
||||||
|
private final Integer maxTotalRows;
|
||||||
|
@Nullable
|
||||||
private final IndexSpec indexSpec;
|
private final IndexSpec indexSpec;
|
||||||
private final int maxPendingPersists;
|
@Nullable
|
||||||
private final long pushTimeout;
|
private final Integer maxPendingPersists;
|
||||||
|
@Nullable
|
||||||
|
private final Long pushTimeout;
|
||||||
|
|
||||||
public static ClientCompactQueryTuningConfig from(
|
public static ClientCompactQueryTuningConfig from(
|
||||||
@Nullable UserCompactTuningConfig userCompactTuningConfig,
|
@Nullable UserCompactTuningConfig userCompactTuningConfig,
|
||||||
|
@ -70,11 +68,11 @@ public class ClientCompactQueryTuningConfig
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.maxRowsPerSegment = maxRowsPerSegment;
|
this.maxRowsPerSegment = maxRowsPerSegment;
|
||||||
this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory;
|
this.maxRowsInMemory = maxRowsInMemory;
|
||||||
this.maxTotalRows = maxTotalRows == null ? DEFAULT_MAX_TOTAL_ROWS : maxTotalRows;
|
this.maxTotalRows = maxTotalRows;
|
||||||
this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.maxPendingPersists = maxPendingPersists == null ? DEFAULT_MAX_PENDING_PERSISTS : maxPendingPersists;
|
this.maxPendingPersists = maxPendingPersists;
|
||||||
this.pushTimeout = pushTimeout == null ? DEFAULT_PUSH_TIMEOUT : pushTimeout;
|
this.pushTimeout = pushTimeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@ -91,31 +89,36 @@ public class ClientCompactQueryTuningConfig
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public int getMaxRowsInMemory()
|
@Nullable
|
||||||
|
public Integer getMaxRowsInMemory()
|
||||||
{
|
{
|
||||||
return maxRowsInMemory;
|
return maxRowsInMemory;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public int getMaxTotalRows()
|
@Nullable
|
||||||
|
public Integer getMaxTotalRows()
|
||||||
{
|
{
|
||||||
return maxTotalRows;
|
return maxTotalRows;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
|
@Nullable
|
||||||
public IndexSpec getIndexSpec()
|
public IndexSpec getIndexSpec()
|
||||||
{
|
{
|
||||||
return indexSpec;
|
return indexSpec;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public int getMaxPendingPersists()
|
@Nullable
|
||||||
|
public Integer getMaxPendingPersists()
|
||||||
{
|
{
|
||||||
return maxPendingPersists;
|
return maxPendingPersists;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public long getPushTimeout()
|
@Nullable
|
||||||
|
public Long getPushTimeout()
|
||||||
{
|
{
|
||||||
return pushTimeout;
|
return pushTimeout;
|
||||||
}
|
}
|
||||||
|
@ -130,25 +133,18 @@ public class ClientCompactQueryTuningConfig
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
ClientCompactQueryTuningConfig that = (ClientCompactQueryTuningConfig) o;
|
ClientCompactQueryTuningConfig that = (ClientCompactQueryTuningConfig) o;
|
||||||
return maxRowsInMemory == that.maxRowsInMemory &&
|
return Objects.equals(maxRowsPerSegment, that.maxRowsPerSegment) &&
|
||||||
maxTotalRows == that.maxTotalRows &&
|
Objects.equals(maxRowsInMemory, that.maxRowsInMemory) &&
|
||||||
maxPendingPersists == that.maxPendingPersists &&
|
Objects.equals(maxTotalRows, that.maxTotalRows) &&
|
||||||
pushTimeout == that.pushTimeout &&
|
Objects.equals(indexSpec, that.indexSpec) &&
|
||||||
Objects.equals(maxRowsPerSegment, that.maxRowsPerSegment) &&
|
Objects.equals(maxPendingPersists, that.maxPendingPersists) &&
|
||||||
Objects.equals(indexSpec, that.indexSpec);
|
Objects.equals(pushTimeout, that.pushTimeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public int hashCode()
|
public int hashCode()
|
||||||
{
|
{
|
||||||
return Objects.hash(
|
return Objects.hash(maxRowsPerSegment, maxRowsInMemory, maxTotalRows, indexSpec, maxPendingPersists, pushTimeout);
|
||||||
maxRowsPerSegment,
|
|
||||||
maxRowsInMemory,
|
|
||||||
maxTotalRows,
|
|
||||||
indexSpec,
|
|
||||||
maxPendingPersists,
|
|
||||||
pushTimeout
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.client.indexing;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.base.Strings;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import org.apache.druid.discovery.DruidLeaderClient;
|
import org.apache.druid.discovery.DruidLeaderClient;
|
||||||
import org.apache.druid.indexer.TaskStatusPlus;
|
import org.apache.druid.indexer.TaskStatusPlus;
|
||||||
|
@ -71,7 +72,7 @@ public class HttpIndexingServiceClient implements IndexingServiceClient
|
||||||
boolean keepSegmentGranularity,
|
boolean keepSegmentGranularity,
|
||||||
@Nullable Long targetCompactionSizeBytes,
|
@Nullable Long targetCompactionSizeBytes,
|
||||||
int compactionTaskPriority,
|
int compactionTaskPriority,
|
||||||
@Nullable ClientCompactQueryTuningConfig tuningConfig,
|
ClientCompactQueryTuningConfig tuningConfig,
|
||||||
@Nullable Map<String, Object> context
|
@Nullable Map<String, Object> context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -103,14 +104,20 @@ public class HttpIndexingServiceClient implements IndexingServiceClient
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
final FullResponseHolder response = druidLeaderClient.go(
|
final FullResponseHolder response = druidLeaderClient.go(
|
||||||
druidLeaderClient.makeRequest(
|
druidLeaderClient.makeRequest(HttpMethod.POST, "/druid/indexer/v1/task")
|
||||||
HttpMethod.POST,
|
.setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject))
|
||||||
"/druid/indexer/v1/task"
|
|
||||||
).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject))
|
|
||||||
);
|
);
|
||||||
|
|
||||||
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
|
if (!response.getStatus().equals(HttpResponseStatus.OK)) {
|
||||||
throw new ISE("Failed to post task[%s]", taskObject);
|
if (!Strings.isNullOrEmpty(response.getContent())) {
|
||||||
|
throw new ISE(
|
||||||
|
"Failed to post task[%s] with error[%s].",
|
||||||
|
taskObject,
|
||||||
|
response.getContent()
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
throw new ISE("Failed to post task[%s]. Please check overlord log", taskObject);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final Map<String, Object> resultMap = jsonMapper.readValue(
|
final Map<String, Object> resultMap = jsonMapper.readValue(
|
||||||
|
@ -132,10 +139,7 @@ public class HttpIndexingServiceClient implements IndexingServiceClient
|
||||||
final FullResponseHolder response = druidLeaderClient.go(
|
final FullResponseHolder response = druidLeaderClient.go(
|
||||||
druidLeaderClient.makeRequest(
|
druidLeaderClient.makeRequest(
|
||||||
HttpMethod.POST,
|
HttpMethod.POST,
|
||||||
StringUtils.format(
|
StringUtils.format("/druid/indexer/v1/task/%s/shutdown", StringUtils.urlEncode(taskId))
|
||||||
"/druid/indexer/v1/task/%s/shutdown",
|
|
||||||
StringUtils.urlEncode(taskId)
|
|
||||||
)
|
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -267,7 +271,10 @@ public class HttpIndexingServiceClient implements IndexingServiceClient
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
final FullResponseHolder responseHolder = druidLeaderClient.go(
|
final FullResponseHolder responseHolder = druidLeaderClient.go(
|
||||||
druidLeaderClient.makeRequest(HttpMethod.GET, StringUtils.format("/druid/indexer/v1/task/%s", taskId))
|
druidLeaderClient.makeRequest(
|
||||||
|
HttpMethod.GET,
|
||||||
|
StringUtils.format("/druid/indexer/v1/task/%s", StringUtils.urlEncode(taskId))
|
||||||
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
return jsonMapper.readValue(
|
return jsonMapper.readValue(
|
||||||
|
|
|
@ -71,10 +71,6 @@ public class DataSourceCompactionConfig
|
||||||
@JsonProperty("taskContext") @Nullable Map<String, Object> taskContext
|
@JsonProperty("taskContext") @Nullable Map<String, Object> taskContext
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
Preconditions.checkArgument(
|
|
||||||
targetCompactionSizeBytes == null || maxRowsPerSegment == null,
|
|
||||||
"targetCompactionSizeBytes and maxRowsPerSegment in tuningConfig can't be used together"
|
|
||||||
);
|
|
||||||
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource");
|
||||||
this.keepSegmentGranularity = keepSegmentGranularity == null
|
this.keepSegmentGranularity = keepSegmentGranularity == null
|
||||||
? DEFAULT_KEEP_SEGMENT_GRANULARITY
|
? DEFAULT_KEEP_SEGMENT_GRANULARITY
|
||||||
|
@ -85,11 +81,11 @@ public class DataSourceCompactionConfig
|
||||||
this.inputSegmentSizeBytes = inputSegmentSizeBytes == null
|
this.inputSegmentSizeBytes = inputSegmentSizeBytes == null
|
||||||
? DEFAULT_INPUT_SEGMENT_SIZE_BYTES
|
? DEFAULT_INPUT_SEGMENT_SIZE_BYTES
|
||||||
: inputSegmentSizeBytes;
|
: inputSegmentSizeBytes;
|
||||||
if (targetCompactionSizeBytes == null && maxRowsPerSegment == null) {
|
this.targetCompactionSizeBytes = getValidTargetCompactionSizeBytes(
|
||||||
this.targetCompactionSizeBytes = DEFAULT_TARGET_COMPACTION_SIZE_BYTES;
|
targetCompactionSizeBytes,
|
||||||
} else {
|
maxRowsPerSegment,
|
||||||
this.targetCompactionSizeBytes = targetCompactionSizeBytes;
|
tuningConfig
|
||||||
}
|
);
|
||||||
this.maxRowsPerSegment = maxRowsPerSegment;
|
this.maxRowsPerSegment = maxRowsPerSegment;
|
||||||
this.maxNumSegmentsToCompact = maxNumSegmentsToCompact == null
|
this.maxNumSegmentsToCompact = maxNumSegmentsToCompact == null
|
||||||
? DEFAULT_NUM_INPUT_SEGMENTS
|
? DEFAULT_NUM_INPUT_SEGMENTS
|
||||||
|
@ -104,6 +100,51 @@ public class DataSourceCompactionConfig
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method is copied from {@code CompactionTask#getValidTargetCompactionSizeBytes}. The only difference is this
|
||||||
|
* method doesn't check 'numShards' which is not supported by {@link UserCompactTuningConfig}.
|
||||||
|
*
|
||||||
|
* Currently, we can't use the same method here because it's in a different module. Until we figure out how to reuse
|
||||||
|
* the same method, this method must be synced with {@code CompactionTask#getValidTargetCompactionSizeBytes}.
|
||||||
|
*/
|
||||||
|
@Nullable
|
||||||
|
private static Long getValidTargetCompactionSizeBytes(
|
||||||
|
@Nullable Long targetCompactionSizeBytes,
|
||||||
|
@Nullable Integer maxRowsPerSegment,
|
||||||
|
@Nullable UserCompactTuningConfig tuningConfig
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (targetCompactionSizeBytes != null) {
|
||||||
|
Preconditions.checkArgument(
|
||||||
|
!hasPartitionConfig(maxRowsPerSegment, tuningConfig),
|
||||||
|
"targetCompactionSizeBytes[%s] cannot be used with maxRowsPerSegment[%s] and maxTotalRows[%s]",
|
||||||
|
targetCompactionSizeBytes,
|
||||||
|
maxRowsPerSegment,
|
||||||
|
tuningConfig == null ? null : tuningConfig.getMaxTotalRows()
|
||||||
|
);
|
||||||
|
return targetCompactionSizeBytes;
|
||||||
|
} else {
|
||||||
|
return hasPartitionConfig(maxRowsPerSegment, tuningConfig) ? null : DEFAULT_TARGET_COMPACTION_SIZE_BYTES;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* his method is copied from {@code CompactionTask#hasPartitionConfig}. The two differences are
|
||||||
|
* 1) this method doesn't check 'numShards' which is not supported by {@link UserCompactTuningConfig}, and
|
||||||
|
* 2) this method accepts an additional 'maxRowsPerSegment' parameter since it's not supported by
|
||||||
|
* {@link UserCompactTuningConfig}.
|
||||||
|
*
|
||||||
|
* Currently, we can't use the same method here because it's in a different module. Until we figure out how to reuse
|
||||||
|
* the same method, this method must be synced with {@code CompactionTask#hasPartitionConfig}.
|
||||||
|
*/
|
||||||
|
private static boolean hasPartitionConfig(
|
||||||
|
@Nullable Integer maxRowsPerSegment,
|
||||||
|
@Nullable UserCompactTuningConfig tuningConfig
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return maxRowsPerSegment != null || (tuningConfig != null && tuningConfig.getMaxTotalRows() != null);
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public String getDataSource()
|
public String getDataSource()
|
||||||
{
|
{
|
||||||
|
|
|
@ -175,7 +175,6 @@ public class DruidCoordinatorSegmentCompactor implements DruidCoordinatorHelper
|
||||||
|
|
||||||
for (; iterator.hasNext() && numSubmittedTasks < numAvailableCompactionTaskSlots; numSubmittedTasks++) {
|
for (; iterator.hasNext() && numSubmittedTasks < numAvailableCompactionTaskSlots; numSubmittedTasks++) {
|
||||||
final List<DataSegment> segmentsToCompact = iterator.next();
|
final List<DataSegment> segmentsToCompact = iterator.next();
|
||||||
|
|
||||||
final String dataSourceName = segmentsToCompact.get(0).getDataSource();
|
final String dataSourceName = segmentsToCompact.get(0).getDataSource();
|
||||||
|
|
||||||
if (segmentsToCompact.size() > 1) {
|
if (segmentsToCompact.size() > 1) {
|
||||||
|
|
|
@ -25,7 +25,9 @@ import org.apache.druid.jackson.DefaultObjectMapper;
|
||||||
import org.apache.druid.server.coordinator.DataSourceCompactionConfig.UserCompactTuningConfig;
|
import org.apache.druid.server.coordinator.DataSourceCompactionConfig.UserCompactTuningConfig;
|
||||||
import org.joda.time.Period;
|
import org.joda.time.Period;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.junit.rules.ExpectedException;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
@ -33,6 +35,9 @@ public class DataSourceCompactionConfigTest
|
||||||
{
|
{
|
||||||
private static final ObjectMapper objectMapper = new DefaultObjectMapper();
|
private static final ObjectMapper objectMapper = new DefaultObjectMapper();
|
||||||
|
|
||||||
|
@Rule
|
||||||
|
public final ExpectedException expectedException = ExpectedException.none();
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testSerdeBasic() throws IOException
|
public void testSerdeBasic() throws IOException
|
||||||
{
|
{
|
||||||
|
@ -56,6 +61,7 @@ public class DataSourceCompactionConfigTest
|
||||||
Assert.assertEquals(25, fromJson.getTaskPriority());
|
Assert.assertEquals(25, fromJson.getTaskPriority());
|
||||||
Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes());
|
Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes());
|
||||||
Assert.assertEquals(config.getTargetCompactionSizeBytes(), fromJson.getTargetCompactionSizeBytes());
|
Assert.assertEquals(config.getTargetCompactionSizeBytes(), fromJson.getTargetCompactionSizeBytes());
|
||||||
|
Assert.assertEquals(config.getMaxRowsPerSegment(), fromJson.getMaxRowsPerSegment());
|
||||||
Assert.assertEquals(config.getMaxNumSegmentsToCompact(), fromJson.getMaxNumSegmentsToCompact());
|
Assert.assertEquals(config.getMaxNumSegmentsToCompact(), fromJson.getMaxNumSegmentsToCompact());
|
||||||
Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest());
|
Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest());
|
||||||
Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig());
|
Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig());
|
||||||
|
@ -85,6 +91,7 @@ public class DataSourceCompactionConfigTest
|
||||||
Assert.assertEquals(25, fromJson.getTaskPriority());
|
Assert.assertEquals(25, fromJson.getTaskPriority());
|
||||||
Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes());
|
Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes());
|
||||||
Assert.assertNull(fromJson.getTargetCompactionSizeBytes());
|
Assert.assertNull(fromJson.getTargetCompactionSizeBytes());
|
||||||
|
Assert.assertEquals(config.getMaxRowsPerSegment(), fromJson.getMaxRowsPerSegment());
|
||||||
Assert.assertEquals(config.getMaxNumSegmentsToCompact(), fromJson.getMaxNumSegmentsToCompact());
|
Assert.assertEquals(config.getMaxNumSegmentsToCompact(), fromJson.getMaxNumSegmentsToCompact());
|
||||||
Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest());
|
Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest());
|
||||||
Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig());
|
Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig());
|
||||||
|
@ -101,4 +108,125 @@ public class DataSourceCompactionConfigTest
|
||||||
final UserCompactTuningConfig fromJson = objectMapper.readValue(json, UserCompactTuningConfig.class);
|
final UserCompactTuningConfig fromJson = objectMapper.readValue(json, UserCompactTuningConfig.class);
|
||||||
Assert.assertEquals(config, fromJson);
|
Assert.assertEquals(config, fromJson);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerdeWithMaxTotalRows() throws IOException
|
||||||
|
{
|
||||||
|
final DataSourceCompactionConfig config = new DataSourceCompactionConfig(
|
||||||
|
"dataSource",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
500L,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
20,
|
||||||
|
new Period(3600),
|
||||||
|
new UserCompactTuningConfig(
|
||||||
|
null,
|
||||||
|
10000,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableMap.of("key", "val")
|
||||||
|
);
|
||||||
|
final String json = objectMapper.writeValueAsString(config);
|
||||||
|
final DataSourceCompactionConfig fromJson = objectMapper.readValue(json, DataSourceCompactionConfig.class);
|
||||||
|
|
||||||
|
Assert.assertEquals(config.getDataSource(), fromJson.getDataSource());
|
||||||
|
Assert.assertTrue(fromJson.isKeepSegmentGranularity());
|
||||||
|
Assert.assertEquals(25, fromJson.getTaskPriority());
|
||||||
|
Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes());
|
||||||
|
Assert.assertNull(fromJson.getTargetCompactionSizeBytes());
|
||||||
|
Assert.assertEquals(config.getMaxRowsPerSegment(), fromJson.getMaxRowsPerSegment());
|
||||||
|
Assert.assertEquals(config.getMaxNumSegmentsToCompact(), fromJson.getMaxNumSegmentsToCompact());
|
||||||
|
Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest());
|
||||||
|
Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig());
|
||||||
|
Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTargetCompactionSizeBytesWithMaxRowsPerSegment()
|
||||||
|
{
|
||||||
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
|
expectedException.expectMessage(
|
||||||
|
"targetCompactionSizeBytes[10000] cannot be used with maxRowsPerSegment[1000] and maxTotalRows[null]"
|
||||||
|
);
|
||||||
|
final DataSourceCompactionConfig config = new DataSourceCompactionConfig(
|
||||||
|
"dataSource",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
500L,
|
||||||
|
10000L,
|
||||||
|
1000,
|
||||||
|
20,
|
||||||
|
new Period(3600),
|
||||||
|
null,
|
||||||
|
ImmutableMap.of("key", "val")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTargetCompactionSizeBytesWithMaxTotalRows()
|
||||||
|
{
|
||||||
|
expectedException.expect(IllegalArgumentException.class);
|
||||||
|
expectedException.expectMessage(
|
||||||
|
"targetCompactionSizeBytes[10000] cannot be used with maxRowsPerSegment[null] and maxTotalRows[10000]"
|
||||||
|
);
|
||||||
|
final DataSourceCompactionConfig config = new DataSourceCompactionConfig(
|
||||||
|
"dataSource",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
500L,
|
||||||
|
10000L,
|
||||||
|
null,
|
||||||
|
20,
|
||||||
|
new Period(3600),
|
||||||
|
new UserCompactTuningConfig(
|
||||||
|
null,
|
||||||
|
10000,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableMap.of("key", "val")
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSerdeMaxTotalRowsWithMaxRowsPerSegment() throws IOException
|
||||||
|
{
|
||||||
|
final DataSourceCompactionConfig config = new DataSourceCompactionConfig(
|
||||||
|
"dataSource",
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
500L,
|
||||||
|
null,
|
||||||
|
10000,
|
||||||
|
20,
|
||||||
|
new Period(3600),
|
||||||
|
new UserCompactTuningConfig(
|
||||||
|
null,
|
||||||
|
10000,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
),
|
||||||
|
ImmutableMap.of("key", "val")
|
||||||
|
);
|
||||||
|
|
||||||
|
final String json = objectMapper.writeValueAsString(config);
|
||||||
|
final DataSourceCompactionConfig fromJson = objectMapper.readValue(json, DataSourceCompactionConfig.class);
|
||||||
|
|
||||||
|
Assert.assertEquals(config.getDataSource(), fromJson.getDataSource());
|
||||||
|
Assert.assertTrue(fromJson.isKeepSegmentGranularity());
|
||||||
|
Assert.assertEquals(25, fromJson.getTaskPriority());
|
||||||
|
Assert.assertEquals(config.getInputSegmentSizeBytes(), fromJson.getInputSegmentSizeBytes());
|
||||||
|
Assert.assertNull(fromJson.getTargetCompactionSizeBytes());
|
||||||
|
Assert.assertEquals(config.getMaxRowsPerSegment(), fromJson.getMaxRowsPerSegment());
|
||||||
|
Assert.assertEquals(config.getMaxNumSegmentsToCompact(), fromJson.getMaxNumSegmentsToCompact());
|
||||||
|
Assert.assertEquals(config.getSkipOffsetFromLatest(), fromJson.getSkipOffsetFromLatest());
|
||||||
|
Assert.assertEquals(config.getTuningConfig(), fromJson.getTuningConfig());
|
||||||
|
Assert.assertEquals(config.getTaskContext(), fromJson.getTaskContext());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue