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
Original file line number Diff line number Diff line change
@@ -0,0 +1,27 @@
/*
* 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.kafka.common.errors.s3;

import org.apache.kafka.common.errors.ApiException;

public class NodeFencedException extends ApiException {

public NodeFencedException(String message) {
super(message);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,7 @@
import org.apache.kafka.common.errors.s3.NodeEpochNotExistException;
import org.apache.kafka.common.errors.s3.CompactedObjectsNotFoundException;
import org.apache.kafka.common.errors.s3.KeyExistException;
import org.apache.kafka.common.errors.s3.NodeFencedException;
import org.apache.kafka.common.errors.s3.ObjectNotExistException;
import org.apache.kafka.common.errors.s3.OffsetNotMatchedException;
import org.apache.kafka.common.errors.s3.RedundantOperationException;
Expand Down Expand Up @@ -400,6 +401,7 @@ public enum Errors {
NODE_EPOCH_NOT_EXIST(511, "The node's epoch does not exist", NodeEpochNotExistException::new),
KEY_EXIST(512, "The key already exists.", KeyExistException::new),
KEY_NOT_EXIST(513, "The key does not exist.", ObjectNotExistException::new),
NODE_FENCED(514, "The node is fenced.", NodeFencedException::new),
STREAM_INNER_ERROR(599, "The stream inner error.", StreamInnerErrorException::new);
// AutoMQ for Kafka inject end

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,12 @@
"type": "[]int64",
"versions": "0+",
"about": "The IDs of the compacted S3 objects"
},
{
"name": "FailoverMode",
"type": "bool",
"versions": "0+",
"about": "The failover mode enabled or not"
}
]
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,12 @@
"type": "int64",
"versions": "0+",
"about": "The node epoch."
},
{
"name": "FailoverMode",
"type": "bool",
"versions": "0+",
"about": "The failover mode enabled or not."
}
]
}
7 changes: 4 additions & 3 deletions core/src/main/scala/kafka/log/stream/s3/ConfigUtils.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ public class ConfigUtils {

public static Config to(KafkaConfig s) {
return new Config()
.nodeId(s.brokerId())
.nodeId(s.nodeId())
.endpoint(s.s3Endpoint())
.region(s.s3Region())
.bucket(s.s3Bucket())
Expand Down Expand Up @@ -56,8 +56,9 @@ public static Config to(KafkaConfig s) {
.mockEnable(s.s3MockEnable())
.objectLogEnable(s.s3ObjectLogEnable())
.networkBaselineBandwidth(s.s3NetworkBaselineBandwidthProp())
.refillPeriodMs(s.s3RefillPeriodMsProp());

.refillPeriodMs(s.s3RefillPeriodMsProp())
.objectRetentionTimeInSecond(s.s3ObjectRetentionTimeInSecond())
.failoverEnable(s.s3FailoverEnable());
}

}
6 changes: 3 additions & 3 deletions core/src/main/scala/kafka/log/stream/s3/DefaultS3Client.java
Original file line number Diff line number Diff line change
Expand Up @@ -43,10 +43,10 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static org.apache.kafka.metadata.stream.S3Config.ACCESS_KEY_NAME;
import static org.apache.kafka.metadata.stream.S3Config.SECRET_KEY_NAME;

public class DefaultS3Client implements Client {
public static final String ACCESS_KEY_NAME = "KAFKA_S3_ACCESS_KEY";
public static final String SECRET_KEY_NAME = "KAFKA_S3_SECRET_KEY";

private final static Logger LOGGER = LoggerFactory.getLogger(DefaultS3Client.class);
private final Config config;
private final StreamMetadataManager metadataManager;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,116 @@
/*
* 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 kafka.log.stream.s3.failover;

import com.automq.stream.s3.failover.FailoverFactory;
import com.automq.stream.s3.metadata.S3ObjectMetadata;
import com.automq.stream.s3.metadata.StreamMetadata;
import com.automq.stream.s3.objects.CommitStreamSetObjectRequest;
import com.automq.stream.s3.objects.CommitStreamSetObjectResponse;
import com.automq.stream.s3.objects.CompactStreamObjectRequest;
import com.automq.stream.s3.objects.ObjectManager;
import com.automq.stream.s3.streams.StreamManager;
import kafka.log.stream.s3.objects.ControllerObjectManager;
import kafka.log.stream.s3.streams.ControllerStreamManager;

import java.util.List;
import java.util.concurrent.CompletableFuture;

public class DefaultFailoverFactory implements FailoverFactory {
private final ControllerStreamManager streamManager;
private final ControllerObjectManager objectManager;

public DefaultFailoverFactory(ControllerStreamManager streamManager, ControllerObjectManager objectManager) {
this.streamManager = streamManager;
this.objectManager = objectManager;
}
@Override
public StreamManager getStreamManager(int nodeId, long epoch) {
return new StreamManager() {
@Override
public CompletableFuture<List<StreamMetadata>> getOpeningStreams() {
return streamManager.getOpeningStreams(nodeId, epoch, true);
}

@Override
public CompletableFuture<List<StreamMetadata>> getStreams(List<Long> list) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<Long> createStream() {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<StreamMetadata> openStream(long streamId, long epoch) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<Void> trimStream(long streamId, long epoch, long offset) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<Void> closeStream(long streamId, long streamEpoch) {
return streamManager.closeStream(streamId, streamEpoch);
}

@Override
public CompletableFuture<Void> deleteStream(long streamId, long epoch) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}
};
}

@Override
public ObjectManager getObjectManager(int nodeId, long epoch) {
return new ObjectManager() {
@Override
public CompletableFuture<Long> prepareObject(int count, long ttl) {
return objectManager.prepareObject(count, ttl);
}

@Override
public CompletableFuture<CommitStreamSetObjectResponse> commitStreamSetObject(CommitStreamSetObjectRequest commitStreamSetObjectRequest) {
return objectManager.commitStreamSetObject(commitStreamSetObjectRequest, nodeId, epoch, true);
}

@Override
public CompletableFuture<Void> compactStreamObject(CompactStreamObjectRequest compactStreamObjectRequest) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<List<S3ObjectMetadata>> getObjects(long streamId, long startOffset, long endOffset, int limit) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<List<S3ObjectMetadata>> getServerObjects() {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}

@Override
public CompletableFuture<List<S3ObjectMetadata>> getStreamObjects(long streamId, long startOffset, long endOffset, int limit) {
return CompletableFuture.failedFuture(new UnsupportedOperationException());
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,10 +29,10 @@
import kafka.log.stream.s3.network.ControllerRequestSender.ResponseHandleResult;
import kafka.log.stream.s3.network.request.WrapRequest;
import kafka.server.KafkaConfig;
import org.apache.kafka.common.message.CommitStreamSetObjectRequestData;
import org.apache.kafka.common.message.CommitStreamSetObjectResponseData;
import org.apache.kafka.common.message.CommitStreamObjectRequestData;
import org.apache.kafka.common.message.CommitStreamObjectResponseData;
import org.apache.kafka.common.message.CommitStreamSetObjectRequestData;
import org.apache.kafka.common.message.CommitStreamSetObjectResponseData;
import org.apache.kafka.common.message.PrepareS3ObjectRequestData;
import org.apache.kafka.common.message.PrepareS3ObjectResponseData;
import org.apache.kafka.common.protocol.ApiKeys;
Expand Down Expand Up @@ -102,6 +102,11 @@ public Builder toRequestBuilder() {

@Override
public CompletableFuture<CommitStreamSetObjectResponse> commitStreamSetObject(CommitStreamSetObjectRequest commitStreamSetObjectRequest) {
return commitStreamSetObject(commitStreamSetObjectRequest, nodeId, nodeId, false);
}

public CompletableFuture<CommitStreamSetObjectResponse> commitStreamSetObject(CommitStreamSetObjectRequest commitStreamSetObjectRequest,
int nodeId, long nodeEpoch, boolean failoverMode) {
CommitStreamSetObjectRequestData request = new CommitStreamSetObjectRequestData()
.setNodeId(nodeId)
.setNodeEpoch(nodeEpoch)
Expand All @@ -114,7 +119,8 @@ public CompletableFuture<CommitStreamSetObjectResponse> commitStreamSetObject(Co
.setStreamObjects(commitStreamSetObjectRequest.getStreamObjects()
.stream()
.map(Convertor::toStreamObjectInRequest).collect(Collectors.toList()))
.setCompactedObjectIds(commitStreamSetObjectRequest.getCompactedObjectIds());
.setCompactedObjectIds(commitStreamSetObjectRequest.getCompactedObjectIds())
.setFailoverMode(failoverMode);
WrapRequest req = new WrapRequest() {
@Override
public ApiKeys apiKey() {
Expand Down
Loading