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

[ISSUE #205] support message body converted into UserDefinedMessage with schema #206

Merged
merged 3 commits into from
Sep 8, 2022
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
5 changes: 5 additions & 0 deletions rocketmq-streams-channel-rocketmq/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,11 @@
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-acl</artifactId>
</dependency>
<dependency>
<groupId>org.apache.rocketmq</groupId>
<artifactId>schema-registry-client</artifactId>
<version>0.0.4-SNAPSHOT</version>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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.streams.schema;

import java.util.HashMap;
import java.util.Map;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.schema.registry.client.SchemaRegistryClient;
import org.apache.rocketmq.schema.registry.client.SchemaRegistryClientFactory;
import org.apache.rocketmq.schema.registry.client.config.AvroSerializerConfig;
import org.apache.rocketmq.schema.registry.client.serde.avro.SpecificAvroSerde;

public class AvroSchemaWrapper implements SchemaWrapper {

private static final Log LOG = LogFactory.getLog(AvroSchemaWrapper.class);

private SpecificAvroSerde avroSerde;

private SchemaConfig schemaConfig;

public AvroSchemaWrapper(SchemaConfig schemaConfig) {
this.schemaConfig = schemaConfig;
try {
if (schemaConfig.getSchemaRegistryUrl() == null) {
avroSerde = new SpecificAvroSerde();
Map<String, Object> configs = new HashMap<>();
configs.put(AvroSerializerConfig.DESERIALIZE_TARGET_TYPE, Class.forName(schemaConfig.getClassName()));
configs.put(AvroSerializerConfig.SKIP_SCHEMA_REGISTRY, true);
avroSerde.configure(configs);
} else {
SchemaRegistryClient schemaRegistryClient =
SchemaRegistryClientFactory.newClient(schemaConfig.getSchemaRegistryUrl(), null);
avroSerde = new SpecificAvroSerde(schemaRegistryClient);
Map<String, Object> configs = new HashMap<>();
configs.put(AvroSerializerConfig.DESERIALIZE_TARGET_TYPE, Class.forName(schemaConfig.getClassName()));
avroSerde.configure(configs);
}
} catch (Exception e) {
LOG.error("init AvroSchemaWrapper failed, " + schemaConfig.toString(), e);
throw new RuntimeException("init AvroSchemaWrapper failed");
}
}

@Override
public Object deserialize(MessageExt messageExt) {
String subject = messageExt.getTopic();
byte[] msgBody = messageExt.getBody();
return avroSerde.deserializer().deserialize(subject, msgBody);
}

@Override
public SchemaConfig getConfig() {
return schemaConfig;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.streams.schema;

import java.util.HashMap;
import java.util.Map;

import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.rocketmq.common.message.MessageExt;
import org.apache.rocketmq.schema.registry.client.SchemaRegistryClient;
import org.apache.rocketmq.schema.registry.client.SchemaRegistryClientFactory;
import org.apache.rocketmq.schema.registry.client.config.JsonSerializerConfig;
import org.apache.rocketmq.schema.registry.client.serde.json.JsonSerde;

/**
* @author huitong
*/
public class JsonSchemaWrapper implements SchemaWrapper {

private static final Log LOG = LogFactory.getLog(JsonSchemaWrapper.class);

private JsonSerde jsonSerde;

private SchemaConfig schemaConfig;

public JsonSchemaWrapper(SchemaConfig schemaConfig) {
this.schemaConfig = schemaConfig;
try {
if (schemaConfig.getSchemaRegistryUrl() == null) {
jsonSerde = new JsonSerde();
Map<String, Object> configs = new HashMap<>();
configs.put(JsonSerializerConfig.DESERIALIZE_TARGET_TYPE, Class.forName(schemaConfig.getClassName()));
configs.put(JsonSerializerConfig.SKIP_SCHEMA_REGISTRY, true);
jsonSerde.configure(configs);
} else {
SchemaRegistryClient schemaRegistryClient =
SchemaRegistryClientFactory.newClient(schemaConfig.getSchemaRegistryUrl(), null);
jsonSerde = new JsonSerde(schemaRegistryClient);
Map<String, Object> configs = new HashMap<>();
configs.put(JsonSerializerConfig.DESERIALIZE_TARGET_TYPE, Class.forName(schemaConfig.getClassName()));
jsonSerde.configure(configs);
}
} catch (Exception e) {
LOG.error("init AvroSchemaWrapper failed, " + schemaConfig.toString(), e);
throw new RuntimeException("init AvroSchemaWrapper failed");
}
}

@Override
public Object deserialize(MessageExt messageExt) {

String subject = messageExt.getTopic();
byte[] msgBody = messageExt.getBody();
return jsonSerde.deserializer().deserialize(subject, msgBody);
}

@Override
public SchemaConfig getConfig() {
return schemaConfig;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
/*
* 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.streams.schema;

import java.io.Serializable;
import org.apache.commons.lang3.StringUtils;

public class SchemaConfig implements Serializable {

/**
* json, avro, protobuf, kyro, thrift, etc.
*/
private String schemaType;

/**
* allowed to be null, if null means skip schema registry
*/
private String schemaRegistryUrl;

/**
* deserialize target class
*/
private String className;

public SchemaConfig() {
}

public SchemaConfig(SchemaType schemaType, Class targetClass) {
this.schemaType = schemaType.name();
this.className = targetClass.getName();
}

public SchemaConfig(SchemaType schemaType, Class targetClass, String schemaRegistryUrl) {
this.schemaType = schemaType.name();
this.schemaRegistryUrl = schemaRegistryUrl;
this.className = targetClass.getName();
}

public String getSchemaType() {
return schemaType;
}

public void setSchemaType(String schemaType) {
this.schemaType = schemaType;
}

public String getSchemaRegistryUrl() {
return schemaRegistryUrl;
}

public void setSchemaRegistryUrl(String schemaRegistryUrl) {
this.schemaRegistryUrl = schemaRegistryUrl;
}

public String getClassName() {
return className;
}

public void setClassName(String className) {
this.className = className;
}

@Override
public String toString() {
return "SchemaConfig{" +
"schemaType='" + schemaType + '\'' +
", schemaRegistryUrl='" + schemaRegistryUrl + '\'' +
", className='" + className + '\'' +
'}';
}

public boolean equals(SchemaConfig configToCompare) {
if (!StringUtils.equals(getSchemaType(), configToCompare.getSchemaType())) {
return false;
}
if (!StringUtils.equals(getClassName(), configToCompare.getClassName())) {
return false;
}
if (!StringUtils.equals(getSchemaRegistryUrl(), configToCompare.getSchemaRegistryUrl())) {
return false;
}
return true;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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.streams.schema;

public enum SchemaType {

/**
* Avro type
*/
AVRO("AVRO"),
/**
* Protobuf type
*/
PROTOBUF("PROTOBUF"),
/**
* Thrift type
*/
THRIFT("THRIFT"),
/**
* Json type
*/
JSON("JSON"),
/**
* Text type for reserved
*/
TEXT("TEXT"),
/**
* Binlog type for reserved
*/
BINLOG("BINLOG");

private final String value;

SchemaType(final String value) {
this.value = value;
}
}
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.rocketmq.streams.schema;

import org.apache.rocketmq.common.message.MessageExt;

public interface SchemaWrapper {

Object deserialize(MessageExt messageExt);

SchemaConfig getConfig();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* 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.streams.schema;

import java.util.HashMap;
import java.util.Map;

public class SchemaWrapperFactory {

private static Map<String /*topic*/, SchemaWrapper> schemaWrapperCache = new HashMap<>();

public static SchemaWrapper createIfAbsent(String topic, SchemaConfig schemaConfig) {
SchemaWrapper schemaWrapper = schemaWrapperCache.get(topic);
if (schemaWrapper != null) {
// if change config, also need to create a new one
if (schemaConfig.equals(schemaWrapper.getConfig())) {
return schemaWrapper;
}
}
if (SchemaType.JSON.name().equals(schemaConfig.getSchemaType())) {
JsonSchemaWrapper jsonSchemaWrapper = new JsonSchemaWrapper(schemaConfig);
schemaWrapperCache.put(topic, jsonSchemaWrapper);
return jsonSchemaWrapper;
} else if (SchemaType.AVRO.name().equals(schemaConfig.getSchemaType())) {
AvroSchemaWrapper avroSchemaWrapper = new AvroSchemaWrapper(schemaConfig);
schemaWrapperCache.put(topic, avroSchemaWrapper);
return avroSchemaWrapper;
} else {
throw new RuntimeException("scheme type " + schemaConfig.getSchemaType() + " not supported");
}
}

}
Loading