Skip to content

Commit

Permalink
[INLONG-11469][SDK] Optimize the single message processing logic in t…
Browse files Browse the repository at this point in the history
…he EncodeObject class (#11470)

Co-authored-by: gosonzhang <[email protected]>
  • Loading branch information
gosonzhang and gosonzhang authored Nov 8, 2024
1 parent d154169 commit 99dec05
Show file tree
Hide file tree
Showing 13 changed files with 182 additions and 309 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ public interface AttributeConstants {

String SEPARATOR = "&";
String KEY_VALUE_SEPARATOR = "=";
String LINE_FEED_SEP = "\n";

/**
* group id
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
Expand Down Expand Up @@ -229,15 +230,6 @@ private String attemptSendMessageIndex(Function<Sender, String> sendOperation) {
return sendIndexResult;
}

@Deprecated
public SendResult sendMessage(byte[] body, String attributes, String msgUUID,
long timeout, TimeUnit timeUnit) {
Function<Sender, SendResult> sendOperation =
(sender) -> sender.syncSendMessage(new EncodeObject(body, attributes, idGenerator.getNextId()), msgUUID,
timeout, timeUnit);
return attemptSendMessage(sendOperation);
}

public SendResult sendMessage(byte[] body, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit) {
return sendMessage(body, groupId, streamId, dt, msgUUID, timeout, timeUnit, false);
Expand Down Expand Up @@ -275,8 +267,9 @@ public SendResult sendMessage(byte[] body, String groupId, String streamId, long
boolean isCompressEnd = (isCompress && (body.length > cpsSize));

if (msgtype == 7 || msgtype == 8) {
EncodeObject encodeObject = new EncodeObject(body, msgtype, isCompressEnd, isReport,
isGroupIdTransfer, dt / 1000, idGenerator.getNextInt(), groupId, streamId, proxySend);
EncodeObject encodeObject =
new EncodeObject(Collections.singletonList(body), msgtype, isCompressEnd, isReport,
isGroupIdTransfer, dt / 1000, idGenerator.getNextInt(), groupId, streamId, proxySend);
encodeObject.setSupportLF(isSupportLF);
Function<Sender, SendResult> sendOperation = (sender) -> sender.syncSendMessage(encodeObject, msgUUID,
timeout, timeUnit);
Expand All @@ -289,13 +282,13 @@ public SendResult sendMessage(byte[] body, String groupId, String streamId, long
final long finalDt = dt;
Function<Sender, SendResult> sendOperation;
if (isCompressEnd) {
sendOperation = (sender) -> sender.syncSendMessage(new EncodeObject(body,
sendOperation = (sender) -> sender.syncSendMessage(new EncodeObject(Collections.singletonList(body),
"groupId=" + groupId + "&streamId=" + streamId + "&dt=" + finalDt + "&cp=snappy"
+ finalProxySend,
idGenerator.getNextId(), this.getMsgtype(),
true, groupId), msgUUID, timeout, timeUnit);
} else {
sendOperation = (sender) -> sender.syncSendMessage(new EncodeObject(body,
sendOperation = (sender) -> sender.syncSendMessage(new EncodeObject(Collections.singletonList(body),
"groupId=" + groupId + "&streamId=" + streamId + "&dt=" + finalDt
+ finalProxySend,
idGenerator.getNextId(), this.getMsgtype(),
Expand Down Expand Up @@ -347,9 +340,10 @@ public SendResult sendMessage(byte[] body, String groupId, String streamId, long
boolean isCompressEnd = (isCompress && (body.length > cpsSize));

if (msgtype == 7 || msgtype == 8) {
EncodeObject encodeObject = new EncodeObject(body, msgtype, isCompressEnd, isReport,
isGroupIdTransfer, dt / 1000,
idGenerator.getNextInt(), groupId, streamId, attrs.toString());
EncodeObject encodeObject =
new EncodeObject(Collections.singletonList(body), msgtype, isCompressEnd, isReport,
isGroupIdTransfer, dt / 1000,
idGenerator.getNextInt(), groupId, streamId, attrs.toString());
encodeObject.setSupportLF(isSupportLF);
Function<Sender, SendResult> sendOperation = (sender) -> sender.syncSendMessage(encodeObject, msgUUID,
timeout, timeUnit);
Expand All @@ -358,13 +352,15 @@ public SendResult sendMessage(byte[] body, String groupId, String streamId, long
attrs.append("&groupId=").append(groupId).append("&streamId=").append(streamId).append("&dt=").append(dt);
if (isCompressEnd) {
attrs.append("&cp=snappy");
Function<Sender, SendResult> sendOperation = (sender) -> sender.syncSendMessage(new EncodeObject(body,
attrs.toString(), idGenerator.getNextId(), this.getMsgtype(), true, groupId),
Function<Sender, SendResult> sendOperation = (sender) -> sender.syncSendMessage(
new EncodeObject(Collections.singletonList(body),
attrs.toString(), idGenerator.getNextId(), this.getMsgtype(), true, groupId),
msgUUID, timeout, timeUnit);
return attemptSendMessage(sendOperation);
} else {
Function<Sender, SendResult> sendOperation = (sender) -> sender.syncSendMessage(new EncodeObject(body,
attrs.toString(), idGenerator.getNextId(), this.getMsgtype(), false, groupId),
Function<Sender, SendResult> sendOperation = (sender) -> sender.syncSendMessage(
new EncodeObject(Collections.singletonList(body),
attrs.toString(), idGenerator.getNextId(), this.getMsgtype(), false, groupId),
msgUUID, timeout, timeUnit);
return attemptSendMessage(sendOperation);
}
Expand Down Expand Up @@ -503,13 +499,6 @@ public SendResult sendMessage(List<byte[]> bodyList, String groupId, String stre
return null;
}

@Deprecated
public void asyncSendMessage(SendMessageCallback callback, byte[] body, String attributes,
String msgUUID, long timeout, TimeUnit timeUnit) throws ProxysdkException {
sender.asyncSendMessage(new EncodeObject(body, attributes, idGenerator.getNextId()),
callback, msgUUID, timeout, timeUnit);
}

public void asyncSendMessage(SendMessageCallback callback, byte[] body, String groupId, String streamId, long dt,
String msgUUID, long timeout, TimeUnit timeUnit) throws ProxysdkException {
asyncSendMessage(callback, body, groupId, streamId, dt, msgUUID, timeout, timeUnit, false);
Expand Down Expand Up @@ -546,23 +535,24 @@ public void asyncSendMessage(SendMessageCallback callback, byte[] body, String g
}
boolean isCompressEnd = (isCompress && (body.length > cpsSize));
if (msgtype == 7 || msgtype == 8) {
EncodeObject encodeObject = new EncodeObject(body, this.getMsgtype(), isCompressEnd, isReport,
isGroupIdTransfer, dt / 1000, idGenerator.getNextInt(),
groupId, streamId, proxySend);
EncodeObject encodeObject =
new EncodeObject(Collections.singletonList(body), this.getMsgtype(), isCompressEnd, isReport,
isGroupIdTransfer, dt / 1000, idGenerator.getNextInt(),
groupId, streamId, proxySend);
encodeObject.setSupportLF(isSupportLF);
sender.asyncSendMessage(encodeObject, callback, msgUUID, timeout, timeUnit);
} else if (msgtype == 3 || msgtype == 5) {
if (isCompressEnd) {
if (isProxySend) {
proxySend = "&" + proxySend;
}
sender.asyncSendMessage(new EncodeObject(body, "groupId="
sender.asyncSendMessage(new EncodeObject(Collections.singletonList(body), "groupId="
+ groupId + "&streamId=" + streamId + "&dt=" + dt + "&cp=snappy" + proxySend,
idGenerator.getNextId(), this.getMsgtype(), true, groupId),
callback, msgUUID, timeout, timeUnit);
} else {
sender.asyncSendMessage(
new EncodeObject(body, "groupId=" + groupId + "&streamId="
new EncodeObject(Collections.singletonList(body), "groupId=" + groupId + "&streamId="
+ streamId + "&dt=" + dt + proxySend, idGenerator.getNextId(),
this.getMsgtype(), false, groupId),
callback,
Expand Down Expand Up @@ -611,21 +601,23 @@ public void asyncSendMessage(SendMessageCallback callback, byte[] body, String g

boolean isCompressEnd = (isCompress && (body.length > cpsSize));
if (msgtype == 7 || msgtype == 8) {
EncodeObject encodeObject = new EncodeObject(body, this.getMsgtype(), isCompressEnd,
isReport, isGroupIdTransfer, dt / 1000, idGenerator.getNextInt(),
groupId, streamId, attrs.toString());
EncodeObject encodeObject =
new EncodeObject(Collections.singletonList(body), this.getMsgtype(), isCompressEnd,
isReport, isGroupIdTransfer, dt / 1000, idGenerator.getNextInt(),
groupId, streamId, attrs.toString());
encodeObject.setSupportLF(isSupportLF);
sender.asyncSendMessage(encodeObject, callback, msgUUID, timeout, timeUnit);
} else if (msgtype == 3 || msgtype == 5) {
attrs.append("&groupId=").append(groupId).append("&streamId=").append(streamId).append("&dt=").append(dt);
if (isCompressEnd) {
attrs.append("&cp=snappy");
sender.asyncSendMessage(new EncodeObject(body, attrs.toString(),
sender.asyncSendMessage(new EncodeObject(Collections.singletonList(body), attrs.toString(),
idGenerator.getNextId(), this.getMsgtype(), true, groupId),
callback, msgUUID, timeout, timeUnit);
} else {
sender.asyncSendMessage(new EncodeObject(body, attrs.toString(), idGenerator.getNextId(),
this.getMsgtype(), false, groupId),
sender.asyncSendMessage(
new EncodeObject(Collections.singletonList(body), attrs.toString(), idGenerator.getNextId(),
this.getMsgtype(), false, groupId),
callback, msgUUID, timeout, timeUnit);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,10 @@ public enum LoadBalance {
WEIGHT_RANDOM("weight random", 3),
WEIGHT_ROBIN("weight robin", 4);

private String name;
private int index;
private final String name;
private final int index;

private LoadBalance(String name, int index) {
LoadBalance(String name, int index) {
this.name = name;
this.index = index;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,25 +27,13 @@

public interface MessageSender {

/**
* This method provides a synchronized function which you want to send data
* with extra attributes except groupId,streamId,dt,etc
* This method is deprecated,we suggest you don't use it.
*
* @param body The data will be sent
* @param attributes The attributes you want to add
*/
@Deprecated
public SendResult sendMessage(byte[] body, String attributes, String msgUUID,
long timeout, TimeUnit timeUnit);

/**
* This method provides a synchronized function which you want to send data without packing
*
* @param body The data will be sent
*
*/
public SendResult sendMessage(byte[] body, String groupId, String streamId, long dt, String msgUUID,
SendResult sendMessage(byte[] body, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit);

/**
Expand All @@ -57,7 +45,7 @@ public SendResult sendMessage(byte[] body, String groupId, String streamId, long
* @param extraAttrMap The attributes you want to add,
* and each element of extraAttrMap contains a pair like attrKey,attrValue
*/
public SendResult sendMessage(byte[] body, String groupId, String streamId, long dt, String msgUUID,
SendResult sendMessage(byte[] body, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap);

/**
Expand All @@ -66,7 +54,7 @@ public SendResult sendMessage(byte[] body, String groupId, String streamId, long
*
* @param bodyList The data will be sent,which is a collection consisting of byte arrays
*/
public SendResult sendMessage(List<byte[]> bodyList, String groupId, String streamId, long dt, String msgUUID,
SendResult sendMessage(List<byte[]> bodyList, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit);

/**
Expand All @@ -78,23 +66,9 @@ public SendResult sendMessage(List<byte[]> bodyList, String groupId, String stre
* @param extraAttrMap The attributes you want to add,
* and each element of extraAttrMap contains a pair like attrKey,attrValue
*/
public SendResult sendMessage(List<byte[]> bodyList, String groupId, String streamId, long dt, String msgUUID,
SendResult sendMessage(List<byte[]> bodyList, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit, Map<String, String> extraAttrMap);

/**
* This method provides an asynchronized function which you want to send data
* with extra attributes except groupId,streamId,dt,etc
* This method is deprecated,we suggest you don't use it.
*
*
* @param body The data will be sent
* @param attributes The attributes you want to add
*/
@Deprecated
public void asyncSendMessage(SendMessageCallback callback,
byte[] body, String attributes, String msgUUID,
long timeout, TimeUnit timeUnit) throws ProxysdkException;

/**
* This method provides a synchronized function which you want to send data without packing
* with extra attributes except groupId,streamId,dt,etc
Expand All @@ -104,7 +78,7 @@ public void asyncSendMessage(SendMessageCallback callback,
* @param extraAttrMap The attributes you want to add,
* and each element of extraAttrMap contains a pair like attrKey,attrValue
*/
public void asyncSendMessage(SendMessageCallback callback,
void asyncSendMessage(SendMessageCallback callback,
byte[] body, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit,
Map<String, String> extraAttrMap) throws ProxysdkException;
Expand All @@ -116,7 +90,7 @@ public void asyncSendMessage(SendMessageCallback callback,
* @param callback The implementation of callback function
* @param body The data will be sent
*/
public void asyncSendMessage(SendMessageCallback callback,
void asyncSendMessage(SendMessageCallback callback,
byte[] body, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit) throws ProxysdkException;

Expand All @@ -126,7 +100,7 @@ public void asyncSendMessage(SendMessageCallback callback,
*
* @param bodyList The data will be sent,which is a collection consisting of byte arrays
*/
public void asyncSendMessage(SendMessageCallback callback,
void asyncSendMessage(SendMessageCallback callback,
List<byte[]> bodyList, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit) throws ProxysdkException;

Expand All @@ -139,7 +113,7 @@ public void asyncSendMessage(SendMessageCallback callback,
* @param extraAttrMap The attributes you want to add, and each
* element of extraAttrMap contains a pair like attrKey,attrValue
*/
public void asyncSendMessage(SendMessageCallback callback,
void asyncSendMessage(SendMessageCallback callback,
List<byte[]> bodyList, String groupId, String streamId, long dt, String msgUUID,
long timeout, TimeUnit timeUnit,
Map<String, String> extraAttrMap) throws ProxysdkException;
Expand Down Expand Up @@ -170,5 +144,5 @@ void asyncSendMessage(String inlongGroupId, String inlongStreamId, byte[] body,
void asyncSendMessage(String inlongGroupId, String inlongStreamId, List<byte[]> bodyList,
SendMessageCallback callback) throws ProxysdkException;

public void close();
void close();
}
Loading

0 comments on commit 99dec05

Please sign in to comment.