Skip to content
Open
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
Expand Up @@ -24,12 +24,16 @@
import apache.rocketmq.v2.EndTransactionResponse;
import apache.rocketmq.v2.ForwardMessageToDeadLetterQueueRequest;
import apache.rocketmq.v2.ForwardMessageToDeadLetterQueueResponse;
import apache.rocketmq.v2.GetOffsetRequest;
import apache.rocketmq.v2.GetOffsetResponse;
import apache.rocketmq.v2.HeartbeatRequest;
import apache.rocketmq.v2.HeartbeatResponse;
import apache.rocketmq.v2.NotifyClientTerminationRequest;
import apache.rocketmq.v2.NotifyClientTerminationResponse;
import apache.rocketmq.v2.QueryAssignmentRequest;
import apache.rocketmq.v2.QueryAssignmentResponse;
import apache.rocketmq.v2.QueryOffsetRequest;
import apache.rocketmq.v2.QueryOffsetResponse;
import apache.rocketmq.v2.QueryRouteRequest;
import apache.rocketmq.v2.QueryRouteResponse;
import apache.rocketmq.v2.RecallMessageRequest;
Expand All @@ -53,6 +57,7 @@
import org.apache.rocketmq.proxy.grpc.v2.common.GrpcClientSettingsManager;
import org.apache.rocketmq.proxy.grpc.v2.consumer.AckMessageActivity;
import org.apache.rocketmq.proxy.grpc.v2.consumer.ChangeInvisibleDurationActivity;
import org.apache.rocketmq.proxy.grpc.v2.consumer.OffsetActivity;
import org.apache.rocketmq.proxy.grpc.v2.consumer.ReceiveMessageActivity;
import org.apache.rocketmq.proxy.grpc.v2.producer.ForwardMessageToDLQActivity;
import org.apache.rocketmq.proxy.grpc.v2.producer.RecallMessageActivity;
Expand All @@ -69,6 +74,7 @@ public class DefaultGrpcMessagingActivity extends AbstractStartAndShutdown imple
protected ReceiveMessageActivity receiveMessageActivity;
protected AckMessageActivity ackMessageActivity;
protected ChangeInvisibleDurationActivity changeInvisibleDurationActivity;
protected OffsetActivity offsetActivity;
protected SendMessageActivity sendMessageActivity;
protected RecallMessageActivity recallMessageActivity;
protected ForwardMessageToDLQActivity forwardMessageToDLQActivity;
Expand All @@ -87,6 +93,7 @@ protected void init(MessagingProcessor messagingProcessor) {
this.receiveMessageActivity = new ReceiveMessageActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
this.ackMessageActivity = new AckMessageActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
this.changeInvisibleDurationActivity = new ChangeInvisibleDurationActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
this.offsetActivity = new OffsetActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
this.sendMessageActivity = new SendMessageActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
this.recallMessageActivity = new RecallMessageActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
this.forwardMessageToDLQActivity = new ForwardMessageToDLQActivity(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
Expand Down Expand Up @@ -118,6 +125,16 @@ public CompletableFuture<QueryAssignmentResponse> queryAssignment(ProxyContext c
return this.routeActivity.queryAssignment(ctx, request);
}

@Override
public CompletableFuture<QueryOffsetResponse> queryOffset(ProxyContext ctx, QueryOffsetRequest request) {
return this.offsetActivity.queryOffset(ctx, request);
}

@Override
public CompletableFuture<GetOffsetResponse> getOffset(ProxyContext ctx, GetOffsetRequest request) {
return this.offsetActivity.getOffset(ctx, request);
}

@Override
public void receiveMessage(ProxyContext ctx, ReceiveMessageRequest request,
StreamObserver<ReceiveMessageResponse> responseObserver) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,12 +25,16 @@
import apache.rocketmq.v2.EndTransactionResponse;
import apache.rocketmq.v2.ForwardMessageToDeadLetterQueueRequest;
import apache.rocketmq.v2.ForwardMessageToDeadLetterQueueResponse;
import apache.rocketmq.v2.GetOffsetRequest;
import apache.rocketmq.v2.GetOffsetResponse;
import apache.rocketmq.v2.HeartbeatRequest;
import apache.rocketmq.v2.HeartbeatResponse;
import apache.rocketmq.v2.NotifyClientTerminationRequest;
import apache.rocketmq.v2.NotifyClientTerminationResponse;
import apache.rocketmq.v2.QueryAssignmentRequest;
import apache.rocketmq.v2.QueryAssignmentResponse;
import apache.rocketmq.v2.QueryOffsetRequest;
import apache.rocketmq.v2.QueryOffsetResponse;
import apache.rocketmq.v2.QueryRouteRequest;
import apache.rocketmq.v2.QueryRouteResponse;
import apache.rocketmq.v2.RecallMessageRequest;
Expand All @@ -57,6 +61,10 @@ public interface GrpcMessagingActivity extends StartAndShutdown {

CompletableFuture<QueryAssignmentResponse> queryAssignment(ProxyContext ctx, QueryAssignmentRequest request);

CompletableFuture<QueryOffsetResponse> queryOffset(ProxyContext ctx, QueryOffsetRequest request);

CompletableFuture<GetOffsetResponse> getOffset(ProxyContext ctx, GetOffsetRequest request);

void receiveMessage(ProxyContext ctx, ReceiveMessageRequest request,
StreamObserver<ReceiveMessageResponse> responseObserver);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,13 +26,17 @@
import apache.rocketmq.v2.EndTransactionResponse;
import apache.rocketmq.v2.ForwardMessageToDeadLetterQueueRequest;
import apache.rocketmq.v2.ForwardMessageToDeadLetterQueueResponse;
import apache.rocketmq.v2.GetOffsetRequest;
import apache.rocketmq.v2.GetOffsetResponse;
import apache.rocketmq.v2.HeartbeatRequest;
import apache.rocketmq.v2.HeartbeatResponse;
import apache.rocketmq.v2.MessagingServiceGrpc;
import apache.rocketmq.v2.NotifyClientTerminationRequest;
import apache.rocketmq.v2.NotifyClientTerminationResponse;
import apache.rocketmq.v2.QueryAssignmentRequest;
import apache.rocketmq.v2.QueryAssignmentResponse;
import apache.rocketmq.v2.QueryOffsetRequest;
import apache.rocketmq.v2.QueryOffsetResponse;
import apache.rocketmq.v2.QueryRouteRequest;
import apache.rocketmq.v2.QueryRouteResponse;
import apache.rocketmq.v2.RecallMessageRequest;
Expand Down Expand Up @@ -303,6 +307,40 @@ public void ackMessage(AckMessageRequest request, StreamObserver<AckMessageRespo
}
}

@Override
public void queryOffset(QueryOffsetRequest request, StreamObserver<QueryOffsetResponse> responseObserver) {
Function<Status, QueryOffsetResponse> statusResponseCreator = status -> QueryOffsetResponse.newBuilder().setStatus(status).build();
ProxyContext context = createContext();
try {
this.addExecutor(this.consumerThreadPoolExecutor,
context,
request,
() -> grpcMessagingActivity.queryOffset(context, request)
.whenComplete((response, throwable) -> writeResponse(context, request, response, responseObserver, throwable, statusResponseCreator)),
responseObserver,
statusResponseCreator);
} catch (Throwable t) {
writeResponse(context, request, null, responseObserver, t, statusResponseCreator);
}
}

@Override
public void getOffset(GetOffsetRequest request, StreamObserver<GetOffsetResponse> responseObserver) {
Function<Status, GetOffsetResponse> statusResponseCreator = status -> GetOffsetResponse.newBuilder().setStatus(status).build();
ProxyContext context = createContext();
try {
this.addExecutor(this.consumerThreadPoolExecutor,
context,
request,
() -> grpcMessagingActivity.getOffset(context, request)
.whenComplete((response, throwable) -> writeResponse(context, request, response, responseObserver, throwable, statusResponseCreator)),
responseObserver,
statusResponseCreator);
} catch (Throwable t) {
writeResponse(context, request, null, responseObserver, t, statusResponseCreator);
}
}

@Override
public void forwardMessageToDeadLetterQueue(ForwardMessageToDeadLetterQueueRequest request,
StreamObserver<ForwardMessageToDeadLetterQueueResponse> responseObserver) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* 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.rocketmq.proxy.grpc.v2.consumer;

import apache.rocketmq.v2.Code;
import apache.rocketmq.v2.GetOffsetRequest;
import apache.rocketmq.v2.GetOffsetResponse;
import apache.rocketmq.v2.QueryOffsetPolicy;
import apache.rocketmq.v2.QueryOffsetRequest;
import apache.rocketmq.v2.QueryOffsetResponse;
import com.google.protobuf.util.Timestamps;
import java.util.concurrent.CompletableFuture;
import org.apache.rocketmq.proxy.common.ProxyContext;
import org.apache.rocketmq.proxy.grpc.v2.AbstractMessagingActivity;
import org.apache.rocketmq.proxy.grpc.v2.channel.GrpcChannelManager;
import org.apache.rocketmq.proxy.grpc.v2.common.GrpcClientSettingsManager;
import org.apache.rocketmq.proxy.grpc.v2.common.GrpcProxyException;
import org.apache.rocketmq.proxy.grpc.v2.common.ResponseBuilder;
import org.apache.rocketmq.proxy.processor.MessagingProcessor;

public class OffsetActivity extends AbstractMessagingActivity {

public OffsetActivity(MessagingProcessor messagingProcessor,
GrpcClientSettingsManager grpcClientSettingsManager, GrpcChannelManager grpcChannelManager) {
super(messagingProcessor, grpcClientSettingsManager, grpcChannelManager);
}

public CompletableFuture<GetOffsetResponse> getOffset(ProxyContext ctx, GetOffsetRequest request) {
CompletableFuture<GetOffsetResponse> future = new CompletableFuture<>();
try {
validateTopicAndConsumerGroup(request.getMessageQueue().getTopic(), request.getGroup());
org.apache.rocketmq.common.message.MessageQueue messageQueue = convertMessageQueue(request.getMessageQueue());
return this.messagingProcessor.queryConsumerOffset(
ctx,
messageQueue,
request.getGroup().getName(),
MessagingProcessor.DEFAULT_TIMEOUT_MILLS
).thenApply(this::convertToGetOffsetResponse);
} catch (Throwable t) {
future.completeExceptionally(t);
}
return future;
}

public CompletableFuture<QueryOffsetResponse> queryOffset(ProxyContext ctx, QueryOffsetRequest request) {
CompletableFuture<QueryOffsetResponse> future = new CompletableFuture<>();
try {
validateTopic(request.getMessageQueue().getTopic());
return queryOffset0(ctx, request).thenApply(this::convertToQueryOffsetResponse);
} catch (Throwable t) {
future.completeExceptionally(t);
}
return future;
}

protected CompletableFuture<Long> queryOffset0(ProxyContext ctx, QueryOffsetRequest request) {
QueryOffsetPolicy queryOffsetPolicy = request.getQueryOffsetPolicy();
org.apache.rocketmq.common.message.MessageQueue messageQueue = convertMessageQueue(request.getMessageQueue());
switch (queryOffsetPolicy) {
case BEGINNING:
return this.messagingProcessor.getMinOffset(
ctx,
messageQueue,
MessagingProcessor.DEFAULT_TIMEOUT_MILLS
);
case END:
return this.messagingProcessor.getMaxOffset(
ctx,
messageQueue,
MessagingProcessor.DEFAULT_TIMEOUT_MILLS
);
case TIMESTAMP:
if (!request.hasTimestamp()) {
throw new GrpcProxyException(Code.BAD_REQUEST, "timestamp is required for TIMESTAMP query offset policy");
}
return this.messagingProcessor.searchOffset(
ctx,
messageQueue,
Timestamps.toMillis(request.getTimestamp()),
MessagingProcessor.DEFAULT_TIMEOUT_MILLS
);
default:
throw new GrpcProxyException(Code.BAD_REQUEST, "unsupported query offset policy: " + queryOffsetPolicy);
}
}

protected org.apache.rocketmq.common.message.MessageQueue convertMessageQueue(
apache.rocketmq.v2.MessageQueue messageQueue) {
return new org.apache.rocketmq.common.message.MessageQueue(
messageQueue.getTopic().getName(),
messageQueue.getBroker().getName(),
messageQueue.getId()
);
}

protected GetOffsetResponse convertToGetOffsetResponse(long offset) {
return GetOffsetResponse.newBuilder()
.setStatus(ResponseBuilder.getInstance().buildStatus(Code.OK, Code.OK.name()))
.setOffset(offset)
.build();
}

protected QueryOffsetResponse convertToQueryOffsetResponse(long offset) {
return QueryOffsetResponse.newBuilder()
.setStatus(ResponseBuilder.getInstance().buildStatus(Code.OK, Code.OK.name()))
.setOffset(offset)
.build();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import org.apache.rocketmq.remoting.protocol.header.PopMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SearchOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.UpdateConsumerOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData;

Expand Down Expand Up @@ -619,6 +620,23 @@ public CompletableFuture<Long> getMinOffset(ProxyContext ctx, MessageQueue messa
return FutureUtils.addExecutor(future, this.executor);
}

public CompletableFuture<Long> searchOffset(ProxyContext ctx, MessageQueue messageQueue, long timestamp,
long timeoutMillis) {
CompletableFuture<Long> future = new CompletableFuture<>();
try {
AddressableMessageQueue addressableMessageQueue = serviceManager.getTopicRouteService()
.buildAddressableMessageQueue(ctx, messageQueue);
SearchOffsetRequestHeader requestHeader = new SearchOffsetRequestHeader();
requestHeader.setTopic(addressableMessageQueue.getTopic());
requestHeader.setQueueId(addressableMessageQueue.getQueueId());
requestHeader.setTimestamp(timestamp);
future = serviceManager.getMessageService().searchOffset(ctx, addressableMessageQueue, requestHeader, timeoutMillis);
} catch (Throwable t) {
future.completeExceptionally(t);
}
return FutureUtils.addExecutor(future, this.executor);
}

protected Set<AddressableMessageQueue> buildAddressableSet(ProxyContext ctx, Set<MessageQueue> mqSet) {
Set<AddressableMessageQueue> addressableMessageQueueSet = new HashSet<>(mqSet.size());
for (MessageQueue mq : mqSet) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -288,6 +288,12 @@ public CompletableFuture<Long> getMinOffset(ProxyContext ctx, MessageQueue messa
return this.consumerProcessor.getMinOffset(ctx, messageQueue, timeoutMillis);
}

@Override
public CompletableFuture<Long> searchOffset(ProxyContext ctx, MessageQueue messageQueue, long timestamp,
long timeoutMillis) {
return this.consumerProcessor.searchOffset(ctx, messageQueue, timestamp, timeoutMillis);
}

@Override
public CompletableFuture<String> recallMessage(ProxyContext ctx, String topic,
String recallHandle, long timeoutMillis) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -345,6 +345,13 @@ CompletableFuture<Long> getMinOffset(
long timeoutMillis
);

CompletableFuture<Long> searchOffset(
ProxyContext ctx,
MessageQueue messageQueue,
long timestamp,
long timeoutMillis
);

CompletableFuture<String> recallMessage(
ProxyContext ctx,
String topic,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@
import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.RecallMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SearchOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.UpdateConsumerOffsetRequestHeader;

Expand Down Expand Up @@ -249,6 +250,16 @@ public CompletableFuture<Long> getMinOffset(ProxyContext ctx, AddressableMessage
);
}

@Override
public CompletableFuture<Long> searchOffset(ProxyContext ctx, AddressableMessageQueue messageQueue,
SearchOffsetRequestHeader requestHeader, long timeoutMillis) {
return this.mqClientAPIFactory.getClient().searchOffset(
messageQueue.getBrokerAddr(),
requestHeader,
timeoutMillis
);
}

@Override
public CompletableFuture<String> recallMessage(ProxyContext ctx, String brokerName,
RecallMessageRequestHeader requestHeader, long timeoutMillis) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@
import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.RecallMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.RecallMessageResponseHeader;
import org.apache.rocketmq.remoting.protocol.header.SearchOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SendMessageResponseHeader;
import org.apache.rocketmq.remoting.protocol.header.UpdateConsumerOffsetRequestHeader;
Expand Down Expand Up @@ -479,6 +480,12 @@ public CompletableFuture<Long> getMinOffset(ProxyContext ctx, AddressableMessage
throw new NotImplementedException("getMinOffset is not implemented in LocalMessageService");
}

@Override
public CompletableFuture<Long> searchOffset(ProxyContext ctx, AddressableMessageQueue messageQueue,
SearchOffsetRequestHeader requestHeader, long timeoutMillis) {
throw new NotImplementedException("searchOffset is not implemented in LocalMessageService");
}

@Override
public CompletableFuture<String> recallMessage(ProxyContext ctx, String brokerName,
RecallMessageRequestHeader requestHeader, long timeoutMillis) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.rocketmq.remoting.protocol.header.PullMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.QueryConsumerOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.RecallMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SearchOffsetRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader;
import org.apache.rocketmq.remoting.protocol.header.UpdateConsumerOffsetRequestHeader;

Expand Down Expand Up @@ -164,6 +165,13 @@ CompletableFuture<Long> getMinOffset(
long timeoutMillis
);

CompletableFuture<Long> searchOffset(
ProxyContext ctx,
AddressableMessageQueue messageQueue,
SearchOffsetRequestHeader requestHeader,
long timeoutMillis
);

CompletableFuture<String> recallMessage(
ProxyContext ctx,
String brokerName,
Expand Down
Loading