Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
4 changes: 3 additions & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,9 @@ docs:

.PHONY: fmt
fmt:
./gradlew :core:spotlessJavaApply :metadata:spotlessJavaApply :storage:spotlessJavaApply :storage:inkless:spotlessJavaApply
./gradlew \
:core:checkstyleMain :core:checkstyleTest :metadata:checkstyleMain :metadata:checkstyleTest :storage:checkstyleMain :storage:checkstyleTest \
:core:spotlessJavaApply :metadata:spotlessJavaApply :storage:spotlessJavaApply :storage:inkless:spotlessJavaApply

.PHONY: test
test:
Expand Down
5 changes: 3 additions & 2 deletions core/src/test/java/kafka/server/InklessClusterTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -206,10 +206,11 @@ public void produceToDisklessAndClassic() throws Exception {
int numRecords = 10;

try (Admin admin = AdminClient.create(clientConfigs)) {
final NewTopic disklessTopic = new NewTopic(disklessTopicName, Map.of(0, List.of(0)))
.configs(Map.of(TopicConfig.DISKLESS_ENABLE_CONFIG, "true"));
final NewTopic classicTopic = new NewTopic(classicTopicName, Map.of(0, List.of(0)))
.configs(Map.of(TopicConfig.DISKLESS_ENABLE_CONFIG, "false"));
// manual assignment is not supported for diskless topics
final NewTopic disklessTopic = new NewTopic(disklessTopicName, 1, (short) 1)
.configs(Map.of(TopicConfig.DISKLESS_ENABLE_CONFIG, "true"));
CreateTopicsResult topics = admin.createTopics(List.of(disklessTopic, classicTopic));
topics.all().get(10, TimeUnit.SECONDS);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -765,7 +765,6 @@ private ApiError createTopic(ControllerRequestContext context,
return new ApiError(Errors.INVALID_REPLICATION_FACTOR,
"Replication factor for diskless topics must be 1 or -1 to use the default value (1).");
}
topic.assignments().clear();
}

if (!topic.assignments().isEmpty()) {
Expand All @@ -779,6 +778,10 @@ private ApiError createTopic(ControllerRequestContext context,
"A manual partition assignment was specified, but numPartitions " +
"was not set to -1.");
}
if (disklessEnabled) {
return new ApiError(INVALID_REQUEST,
"A manual partition assignment cannot be specified for diskless topics.");
}
OptionalInt replicationFactor = OptionalInt.empty();
for (CreatableReplicaAssignment assignment : topic.assignments()) {
if (newParts.containsKey(assignment.partitionIndex())) {
Expand Down
Loading