generated from streamnative/pulsar-io-template
-
Notifications
You must be signed in to change notification settings - Fork 20
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Support bytes schema for the lakehouse connector (#75)
--- Fixes: #60 *Motivation* Support bytes schema for the lakehouse connector. We construct the PulsarObject for saving the primitive schema(currently only have bytes and string). The message with primitive schema will write into the lakehouse table with key name 'message' by default. You can change the name according to the configuration. (cherry picked from commit 946ef80)
- Loading branch information
Showing
8 changed files
with
376 additions
and
2 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
50 changes: 50 additions & 0 deletions
50
src/main/java/org/apache/pulsar/ecosystem/io/lakehouse/sink/PrimitiveFactory.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
/** | ||
* 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.pulsar.ecosystem.io.lakehouse.sink; | ||
|
||
import java.nio.ByteBuffer; | ||
import org.apache.avro.Schema; | ||
import org.apache.commons.lang.StringUtils; | ||
import org.apache.pulsar.common.schema.SchemaType; | ||
|
||
/** | ||
* PrimitiveFactory provides a way to get different PulsarObject according to the given schema type. | ||
*/ | ||
public class PrimitiveFactory { | ||
|
||
public static PulsarObject getPulsarPrimitiveObject(SchemaType schemaType, Object value, | ||
String overrideFieldName) { | ||
PulsarObject object; | ||
switch (schemaType) { | ||
case BYTES: | ||
object = new PulsarObject<>(ByteBuffer.wrap((byte[]) value), Schema.create(Schema.Type.BYTES)); | ||
break; | ||
case STRING: | ||
object = new PulsarObject<>((String) value, Schema.create(Schema.Type.STRING)); | ||
break; | ||
default: | ||
throw new RuntimeException("Failed to build pulsar object, the given type '" + schemaType + "' " | ||
+ "is not supported yet."); | ||
} | ||
if (StringUtils.isNotEmpty(overrideFieldName)) { | ||
object.overrideFieldName(overrideFieldName); | ||
} | ||
return object; | ||
} | ||
} |
81 changes: 81 additions & 0 deletions
81
src/main/java/org/apache/pulsar/ecosystem/io/lakehouse/sink/PulsarObject.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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.pulsar.ecosystem.io.lakehouse.sink; | ||
|
||
import java.util.UUID; | ||
import lombok.EqualsAndHashCode; | ||
import org.apache.avro.Schema; | ||
import org.apache.avro.SchemaBuilder; | ||
import org.apache.avro.generic.GenericData; | ||
import org.apache.avro.generic.GenericRecord; | ||
|
||
/** | ||
* The pulsar object represents the pulsar message with primitive schemas. | ||
* This object will be serialized with Avro then writing to the lakehouse table. | ||
* | ||
* @param <T> is the type of the given value. | ||
*/ | ||
@EqualsAndHashCode | ||
public class PulsarObject<T> { | ||
|
||
private static String defaultFieldname = "message"; | ||
private final Schema valueSchema; | ||
T value; | ||
String uuid; | ||
|
||
public PulsarObject(T value, Schema schema) { | ||
this.value = value; | ||
valueSchema = schema; | ||
this.uuid = UUID.randomUUID().toString(); | ||
} | ||
|
||
private PulsarObject(T value, Schema schema, String uuid) { | ||
this.value = value; | ||
valueSchema = schema; | ||
this.uuid = uuid; | ||
} | ||
|
||
public static void overrideFieldName(String fieldName) { | ||
defaultFieldname = fieldName; | ||
} | ||
|
||
public Schema getSchema() { | ||
return SchemaBuilder.record("PulsarObject") | ||
.fields() | ||
.name(defaultFieldname).type(valueSchema).noDefault() | ||
.name("uuid").type(Schema.create(Schema.Type.STRING)).noDefault() | ||
.endRecord(); | ||
} | ||
|
||
public GenericRecord getRecord() { | ||
GenericRecord record = new GenericData.Record(getSchema()); | ||
record.put(defaultFieldname, value); | ||
record.put("uuid", uuid); | ||
return record; | ||
} | ||
|
||
public static <T> PulsarObject<T> fromGenericRecord(GenericRecord record) { | ||
if (!record.hasField(defaultFieldname) && !record.hasField("uuid")) { | ||
throw new RuntimeException("Unexpected record when parsing to the PulsarObject"); | ||
} | ||
return new PulsarObject(record.get(defaultFieldname), | ||
record.getSchema().getField(defaultFieldname).schema() | ||
, record.get("uuid").toString()); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
77 changes: 77 additions & 0 deletions
77
src/test/java/org/apache/pulsar/ecosystem/io/lakehouse/SinkConnectorConfigTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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.pulsar.ecosystem.io.lakehouse; | ||
|
||
import static org.testng.AssertJUnit.assertEquals; | ||
import static org.testng.AssertJUnit.fail; | ||
import java.io.IOException; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
import org.apache.pulsar.ecosystem.io.lakehouse.exception.IncorrectParameterException; | ||
import org.testng.annotations.Test; | ||
|
||
public class SinkConnectorConfigTest { | ||
|
||
@Test | ||
public void testDefaultValues() throws Exception { | ||
Map<String, Object> properties = new HashMap<>(); | ||
properties.put("type", "hudi"); | ||
SinkConnectorConfig config = SinkConnectorConfig.load(properties); | ||
assertEquals(SinkConnectorConfig.DEFAULT_MAX_COMMIT_INTERVAL, config.getMaxCommitInterval()); | ||
assertEquals(SinkConnectorConfig.DEFAULT_MAX_RECORDS_PER_COMMIT, config.getMaxRecordsPerCommit()); | ||
assertEquals(SinkConnectorConfig.DEFAULT_MAX_COMMIT_FAILED_TIMES, config.getMaxCommitFailedTimes()); | ||
assertEquals(SinkConnectorConfig.DEFAULT_SINK_CONNECTOR_QUEUE_SIZE, config.getSinkConnectorQueueSize()); | ||
assertEquals(Collections.emptyList(), config.getPartitionColumns()); | ||
assertEquals("", config.getOverrideFieldName()); | ||
} | ||
|
||
@Test | ||
public void testOverrideDefaultValues() throws Exception { | ||
Map<String, Object> properties = new HashMap<>(); | ||
properties.put("type", "hudi"); | ||
properties.put("partitionColumns", Collections.singletonList("partition")); | ||
properties.put("maxCommitInterval", 10); | ||
properties.put("maxRecordsPerCommit", 10); | ||
properties.put("maxCommitFailedTimes", 10); | ||
properties.put("sinkConnectorQueueSize", 10); | ||
properties.put("overrideFieldName", "filedname"); | ||
SinkConnectorConfig config = SinkConnectorConfig.load(properties); | ||
assertEquals(10, config.getMaxCommitInterval()); | ||
assertEquals(10, config.getMaxRecordsPerCommit()); | ||
assertEquals(10, config.getMaxCommitFailedTimes()); | ||
assertEquals(10, config.getSinkConnectorQueueSize()); | ||
assertEquals(Collections.singletonList("partition"), config.getPartitionColumns()); | ||
assertEquals("filedname", config.getOverrideFieldName()); | ||
} | ||
|
||
@Test | ||
public void testLoadInvalidType() { | ||
Map<String, Object> properties = new HashMap<>(); | ||
properties.put("type", "unknown"); | ||
try { | ||
SinkConnectorConfig.load(properties); | ||
fail(); | ||
} catch (IOException e) { | ||
throw new RuntimeException(e); | ||
} catch (IncorrectParameterException e) { | ||
// expected exception | ||
} | ||
} | ||
} |
56 changes: 56 additions & 0 deletions
56
src/test/java/org/apache/pulsar/ecosystem/io/lakehouse/sink/PrimitiveFactoryTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,56 @@ | ||
/** | ||
* 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.pulsar.ecosystem.io.lakehouse.sink; | ||
|
||
import static org.testng.Assert.assertEquals; | ||
import static org.testng.Assert.assertTrue; | ||
import java.nio.ByteBuffer; | ||
import java.nio.charset.StandardCharsets; | ||
import org.apache.avro.Schema; | ||
import org.apache.pulsar.common.schema.SchemaType; | ||
import org.testng.annotations.Test; | ||
|
||
public class PrimitiveFactoryTest { | ||
|
||
@Test(expectedExceptions = RuntimeException.class) | ||
public void testUnsupportedType() { | ||
PrimitiveFactory.getPulsarPrimitiveObject(SchemaType.AVRO, null, ""); | ||
} | ||
|
||
@Test | ||
public void testPrimitiveBytes() { | ||
byte[] message = "test".getBytes(StandardCharsets.UTF_8); | ||
PulsarObject object = PrimitiveFactory.getPulsarPrimitiveObject(SchemaType.BYTES, message, ""); | ||
Object value = object.getRecord().get("message"); | ||
assertTrue(value instanceof ByteBuffer); | ||
assertEquals(Schema.Type.BYTES, object.getSchema().getField("message").schema().getType()); | ||
ByteBuffer byteBufferValue = (ByteBuffer) value; | ||
assertEquals(byteBufferValue.array(), message); | ||
} | ||
|
||
@Test | ||
public void testPrimitiveString() { | ||
String message = "test"; | ||
PulsarObject object = PrimitiveFactory.getPulsarPrimitiveObject(SchemaType.STRING, message, ""); | ||
Object value = object.getRecord().get("message"); | ||
assertTrue(value instanceof String); | ||
assertEquals(Schema.Type.STRING, object.getSchema().getField("message").schema().getType()); | ||
assertEquals(value.toString(), message); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.