Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[INLONG-11252][SDK] Transform support PB sink data #11280

Merged
merged 3 commits into from
Oct 10, 2024
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,115 @@
/*
* 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.inlong.sdk.transform.encode;

import org.apache.inlong.sdk.transform.pojo.FieldInfo;
import org.apache.inlong.sdk.transform.pojo.PbSinkInfo;
import org.apache.inlong.sdk.transform.process.Context;

import com.google.protobuf.ByteString;
import com.google.protobuf.DescriptorProtos;
import com.google.protobuf.Descriptors;
import com.google.protobuf.DynamicMessage;

import java.util.Base64;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

public class PbSinkEncoder implements SinkEncoder<byte[]> {

protected PbSinkInfo sinkInfo;

private Descriptors.Descriptor dynamicDescriptor;

private final Map<String, Descriptors.FieldDescriptor.Type> fieldTypes;

public PbSinkEncoder(PbSinkInfo pbSinkInfo) {
this.sinkInfo = pbSinkInfo;
this.fieldTypes = new HashMap<>();
for (FieldInfo field : pbSinkInfo.getFields()) {
fieldTypes.put(field.getName(), Descriptors.FieldDescriptor.Type.STRING);
}
// decode protoDescription
this.dynamicDescriptor = decodeProtoDescription(pbSinkInfo.getProtoDescription());
}

@Override
public byte[] encode(SinkData sinkData, Context context) {
try {
DynamicMessage.Builder dynamicBuilder = DynamicMessage.newBuilder(dynamicDescriptor);
luchunliang marked this conversation as resolved.
Show resolved Hide resolved

// Dynamically fill message fields
for (String key : sinkData.keyList()) {
Descriptors.FieldDescriptor fieldDescriptor = dynamicDescriptor.findFieldByName(key);
if (fieldDescriptor != null) {
String fieldValue = sinkData.getField(key);
if (fieldValue != null) {
Object value = convertValue(fieldDescriptor, fieldValue);
dynamicBuilder.setField(fieldDescriptor, value);
}
}
}
// Serialize to byte[]
return dynamicBuilder.build().toByteArray();
} catch (Exception e) {
throw new RuntimeException(e);
}
}

/**
* Decode the base64-encoded proto description into a Descriptor
*
* @param base64ProtoDescription The base64-encoded proto description
* @return The dynamic Descriptor
*/
private Descriptors.Descriptor decodeProtoDescription(String base64ProtoDescription) {
try {
byte[] protoBytes = Base64.getDecoder().decode(base64ProtoDescription);
DescriptorProtos.FileDescriptorSet fileDescriptorSet =
DescriptorProtos.FileDescriptorSet.parseFrom(protoBytes);
Descriptors.FileDescriptor fileDescriptor = Descriptors.FileDescriptor.buildFrom(
fileDescriptorSet.getFile(0), new Descriptors.FileDescriptor[]{});
return fileDescriptor.getMessageTypes().get(0);
} catch (Exception e) {
throw new RuntimeException("Failed to decode protoDescription", e);
}
}

private Object convertValue(Descriptors.FieldDescriptor fieldDescriptor, Object value) {
switch (fieldDescriptor.getType()) {
case STRING:
return value.toString();
case INT32:
return Integer.parseInt(value.toString());
case INT64:
return Long.parseLong(value.toString());
case BOOL:
return Boolean.parseBoolean(value.toString());
case BYTES:
return ByteString.copyFromUtf8(value.toString());
default:
throw new IllegalArgumentException("Unsupported field type: " + fieldDescriptor.getType());
}
}

@Override
public List<FieldInfo> getFields() {
return sinkInfo.getFields();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.inlong.sdk.transform.pojo.KvSinkInfo;
import org.apache.inlong.sdk.transform.pojo.MapSinkInfo;
import org.apache.inlong.sdk.transform.pojo.ParquetSinkInfo;
import org.apache.inlong.sdk.transform.pojo.PbSinkInfo;

public class SinkEncoderFactory {

Expand All @@ -39,4 +40,9 @@ public static MapSinkEncoder createMapEncoder(MapSinkInfo mapSinkInfo) {
public static ParquetSinkEncoder createParquetEncoder(ParquetSinkInfo parquetSinkInfo) {
return new ParquetSinkEncoder(parquetSinkInfo);
}

public static PbSinkEncoder createPbEncoder(PbSinkInfo pbSinkInfo) {
return new PbSinkEncoder(pbSinkInfo);
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
/*
* 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.inlong.sdk.transform.pojo;

import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import com.fasterxml.jackson.annotation.JsonProperty;
import lombok.Data;
import lombok.experimental.SuperBuilder;
import org.apache.commons.collections.CollectionUtils;

import java.util.List;

@JsonIgnoreProperties(ignoreUnknown = true)
@SuperBuilder
@Data
public class PbSinkInfo extends SinkInfo {

private List<FieldInfo> fields;
private String protoDescription;

public PbSinkInfo(
@JsonProperty("charset") String charset,
@JsonProperty("protoDescription") String protoDescription,
@JsonProperty("fields") List<FieldInfo> fields) {
super(SinkInfo.PB, charset);
if (CollectionUtils.isEmpty(fields)) {
throw new IllegalArgumentException("failed to init pb sink info, fieldInfos is empty");
}
this.protoDescription = protoDescription;
this.fields = fields;
}

/**
* get fields
* @return the fields
*/
@JsonProperty("fields")
public List<FieldInfo> getFields() {
return fields;
}

/**
* set fields
* @param fields the fields to set
*/
public void setFields(List<FieldInfo> fields) {
this.fields = fields;
}

/**
* get protoDescription
* @return the protoDescription
*/
@JsonProperty("protoDescription")
public String getProtoDescription() {
return protoDescription;
}

/**
* set protoDescription
* @param protoDescription the protoDescription to set
*/
public void setProtoDescription(String protoDescription) {
this.protoDescription = protoDescription;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ public abstract class SinkInfo {
public static final String KV = "kv";
public static final String ES_MAP = "es_map";
public static final String PARQUET = "parquet";
public static final String PB = "pb";

@JsonIgnore
private String type;
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* 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.inlong.sdk.transform.process.processor;

import org.apache.inlong.sdk.transform.decode.SourceDecoderFactory;
import org.apache.inlong.sdk.transform.encode.SinkEncoderFactory;
import org.apache.inlong.sdk.transform.pojo.CsvSourceInfo;
import org.apache.inlong.sdk.transform.pojo.FieldInfo;
import org.apache.inlong.sdk.transform.pojo.JsonSourceInfo;
import org.apache.inlong.sdk.transform.pojo.KvSourceInfo;
import org.apache.inlong.sdk.transform.pojo.PbSinkInfo;
import org.apache.inlong.sdk.transform.pojo.PbSourceInfo;
import org.apache.inlong.sdk.transform.pojo.TransformConfig;
import org.apache.inlong.sdk.transform.process.TransformProcessor;

import org.junit.Assert;
import org.junit.Test;

import java.util.HashMap;
import java.util.List;

public class TestAny2PbProcessor extends AbstractProcessorTestBase {

@Test
public void testPb2Pb() throws Exception {
List<FieldInfo> fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg");
String transformBase64 = this.getPbTestDescription();
PbSourceInfo pbSource = new PbSourceInfo("UTF-8", transformBase64, "SdkDataRequest", "msgs");
PbSinkInfo pbSink = new PbSinkInfo("UTF-8", transformBase64, fields);
String transformSql = "select $root.sid,$root.packageID,$child.msgTime,$child.msg from source";
TransformConfig config = new TransformConfig(transformSql);
TransformProcessor<byte[], byte[]> processor = TransformProcessor
.create(config, SourceDecoderFactory.createPbDecoder(pbSource),
SinkEncoderFactory.createPbEncoder(pbSink));
byte[] srcBytes = this.getPbTestData();
List<byte[]> output = processor.transform(srcBytes);
Assert.assertEquals(2, output.size());

// case1:
// encode pb: {test.SdkMessage.msg=<ByteString@2a7f1f10 size=9 contents="msgValue4">,
// test.SdkMessage.msgTime=1713243918000}
// result output: [10, 9, 109, 115, 103, 86, 97, 108, 117, 101, 52, 16,-80, -99, -82, -86, -18, 49]

byte[] res1 = {10, 9, 109, 115, 103, 86, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49};
Assert.assertArrayEquals(output.get(0), res1);

// case2:
// encode pb: {test.SdkMessage.msg=<ByteString@7526515b size=10 contents="msgValue42">,
// test.SdkMessage.msgTime=1713243918002}
// result output: [10, 10, 109, 115, 103, 86, 97, 108, 117, 101, 52, 50, 16, -78, -99, -82, -86, -18, 49]

byte[] res2 = {10, 10, 109, 115, 103, 86, 97, 108, 117, 101, 52, 50, 16, -78, -99, -82, -86, -18, 49};
Assert.assertArrayEquals(output.get(1), res2);
}

@Test
public void testCsv2PbForOne() throws Exception {
List<FieldInfo> fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg");
String transformBase64 = this.getPbTestDescription();
CsvSourceInfo csvSource = new CsvSourceInfo("UTF-8", '|', '\\', fields);
PbSinkInfo pbSink = new PbSinkInfo("UTF-8", transformBase64, fields);
String transformSql = "select sid,packageID,msgTime,msg from source";
TransformConfig config = new TransformConfig(transformSql);
TransformProcessor<String, byte[]> processor = TransformProcessor
.create(config, SourceDecoderFactory.createCsvDecoder(csvSource),
SinkEncoderFactory.createPbEncoder(pbSink));
List<byte[]> output = processor.transform("sid|1|1713243918000|msgValue4");
Assert.assertEquals(1, output.size());

// case1:
// encode pb: {test.SdkMessage.msg=<ByteString@6574a52c size=9 contents="msgValue4">,
// test.SdkMessage.msgTime=1713243918000}
// result output: [10, 9, 109, 115, 103, 86, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49]

byte[] res = {10, 9, 109, 115, 103, 86, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49};
Assert.assertArrayEquals(output.get(0), res);
}

@Test
public void testKv2Pb() throws Exception {
List<FieldInfo> fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg");
KvSourceInfo kvSource = new KvSourceInfo("UTF-8", fields);
String transformBase64 = this.getPbTestDescription();
PbSinkInfo pbSink = new PbSinkInfo("UTF-8", transformBase64, fields);
String transformSql = "select sid,packageID,msgTime,msg from source";
TransformConfig config = new TransformConfig(transformSql);
TransformProcessor<String, byte[]> processor = TransformProcessor
.create(config, SourceDecoderFactory.createKvDecoder(kvSource),
SinkEncoderFactory.createPbEncoder(pbSink));
List<byte[]> output =
processor.transform("sid=sid&packageID=1&msgTime=1713243918000&msg=msgValue4", new HashMap<>());
Assert.assertEquals(1, output.size());

// case1:
// encode pb: {test.SdkMessage.msg=<ByteString@6574a52c size=9 contents="msgValue4">,
// test.SdkMessage.msgTime=1713243918000}
// result output: [10, 9, 109, 115, 103, 86, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49]

byte[] res = {10, 9, 109, 115, 103, 86, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49};
Assert.assertArrayEquals(output.get(0), res);
}

@Test
public void testJson2Pb() throws Exception {
List<FieldInfo> fields = this.getTestFieldList("sid", "packageID", "msgTime", "msg");
JsonSourceInfo jsonSource = new JsonSourceInfo("UTF-8", "msgs");
String transformBase64 = this.getPbTestDescription();
PbSinkInfo pbSink = new PbSinkInfo("UTF-8", transformBase64, fields);
String transformSql = "select $root.sid,$root.packageID,$child.msgTime,$child.msg from source";
TransformConfig config = new TransformConfig(transformSql);
TransformProcessor<String, byte[]> processor = TransformProcessor
.create(config, SourceDecoderFactory.createJsonDecoder(jsonSource),
SinkEncoderFactory.createPbEncoder(pbSink));
String srcString1 = "{\n"
+ " \"sid\":\"value1\",\n"
+ " \"packageID\":\"value2\",\n"
+ " \"msgs\":[\n"
+ " {\"msg\":\"value4\",\"msgTime\":1713243918000},\n"
+ " {\"msg\":\"v4\",\"msgTime\":1713243918000}\n"
+ " ]\n"
+ "}";
List<byte[]> output = processor.transform(srcString1, new HashMap<>());
Assert.assertEquals(2, output.size());

// case1:
// encode pb: {test.SdkMessage.msg=<ByteString@7bd7d6d6 size=6 contents="value4">,
// test.SdkMessage.msgTime=1713243918000}
// result output: [10, 6, 118, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49]

byte[] res1 = {10, 6, 118, 97, 108, 117, 101, 52, 16, -80, -99, -82, -86, -18, 49};
Assert.assertArrayEquals(output.get(0), res1);

// case1:
// encode pb: {test.SdkMessage.msg=<ByteString@5745ca0e size=2 contents="v4">,
// test.SdkMessage.msgTime=1713243918000}
// result output: [10, 2, 118, 52, 16, -80, -99, -82, -86, -18, 49]

byte[] res2 = {10, 2, 118, 52, 16, -80, -99, -82, -86, -18, 49};
Assert.assertArrayEquals(output.get(1), res2);
}

}
Loading