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

[feature][functions]Create a built-in Function implementing the most common basic transformations #15903

Closed
wants to merge 20 commits into from
Closed
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
2 changes: 2 additions & 0 deletions pulsar-functions/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@
<module>secrets</module>
<module>localrun</module>
<module>localrun-shaded</module>
<module>transforms</module>
</modules>
</profile>

Expand All @@ -69,6 +70,7 @@
<module>worker</module>
<module>secrets</module>
<module>localrun</module>
<module>transforms</module>
</modules>
</profile>

Expand Down
78 changes: 78 additions & 0 deletions pulsar-functions/transforms/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
<!--

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.

-->
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.pulsar</groupId>
<artifactId>pulsar-functions</artifactId>
<version>2.11.0-SNAPSHOT</version>
</parent>

<artifactId>pulsar-functions-transforms</artifactId>
<name>Pulsar Functions :: Transforms</name>

<dependencies>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-api</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-functions-api</artifactId>
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>${avro.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>com.google.code.gson</groupId>
<artifactId>gson</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.commons</groupId>
<artifactId>commons-lang3</artifactId>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>${project.groupId}</groupId>
<artifactId>pulsar-client-original</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.nifi</groupId>
<artifactId>nifi-nar-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/**
* 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.functions.transforms;

import lombok.extern.slf4j.Slf4j;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.schema.SchemaType;

@Slf4j
public class CastStep implements TransformStep {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the purpose if this Step is not clear to me.
what does this step do ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It changes the schema to another compatible one. The name is inspired from SMTs


private final SchemaType keySchemaType;
private final SchemaType valueSchemaType;

public CastStep(SchemaType keySchemaType, SchemaType valueSchemaType) {
if (keySchemaType != null && keySchemaType != SchemaType.STRING) {
throw new IllegalArgumentException("Unsupported key schema-type for Cast: " + keySchemaType);
}
if (valueSchemaType != null && valueSchemaType != SchemaType.STRING) {
throw new IllegalArgumentException("Unsupported value schema-type for Cast: " + valueSchemaType);
}
this.keySchemaType = keySchemaType;
this.valueSchemaType = valueSchemaType;
}

@Override
public void process(TransformContext transformContext) {
if (transformContext.getKeySchema() != null) {
Object outputKeyObject = transformContext.getKeyObject();
Schema<?> outputSchema = transformContext.getKeySchema();
if (keySchemaType == SchemaType.STRING) {
outputSchema = Schema.STRING;
outputKeyObject = outputKeyObject.toString();
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

else ?
we should throw a IllegalStateException here if we are not supporting other types

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

transformContext.setKeySchema(outputSchema);
transformContext.setKeyObject(outputKeyObject);
}
if (valueSchemaType == SchemaType.STRING) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

else ?
we should throw a IllegalStateException here if we are not supporting other types

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

transformContext.setValueSchema(Schema.STRING);
transformContext.setValueObject(transformContext.getValueObject().toString());
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,103 @@
/**
* 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.functions.transforms;

import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import lombok.extern.slf4j.Slf4j;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.pulsar.common.schema.SchemaType;


/**
* This function removes a "field" from a message.
*/
@Slf4j
public class DropFieldStep implements TransformStep {

private final List<String> keyFields;
private final List<String> valueFields;
private final Map<org.apache.avro.Schema, org.apache.avro.Schema> keySchemaCache = new ConcurrentHashMap<>();
private final Map<org.apache.avro.Schema, org.apache.avro.Schema> valueSchemaCache = new ConcurrentHashMap<>();

public DropFieldStep(List<String> keyFields, List<String> valueFields) {
this.keyFields = keyFields;
this.valueFields = valueFields;
}

@Override
public void process(TransformContext transformContext) {
dropKeyFields(keyFields, transformContext);
dropValueFields(valueFields, transformContext);
}

public void dropValueFields(List<String> fields, TransformContext record) {
if (record.getValueSchema().getSchemaInfo().getType() == SchemaType.AVRO) {
GenericRecord avroRecord = (GenericRecord) record.getValueObject();
GenericRecord newRecord = dropFields(fields, avroRecord, valueSchemaCache);
if (avroRecord != newRecord) {
record.setValueModified(true);
}
record.setValueObject(newRecord);
}
}

public void dropKeyFields(List<String> fields, TransformContext record) {
if (record.getKeyObject() != null && record.getValueSchema().getSchemaInfo().getType() == SchemaType.AVRO) {
GenericRecord avroRecord = (GenericRecord) record.getKeyObject();
GenericRecord newRecord = dropFields(fields, avroRecord, keySchemaCache);
if (avroRecord != newRecord) {
record.setKeyModified(true);
}
record.setKeyObject(newRecord);
}
}

private GenericRecord dropFields(
List<String> fields,
GenericRecord record,
Map<org.apache.avro.Schema, org.apache.avro.Schema> schemaCache
) {
org.apache.avro.Schema avroSchema = record.getSchema();
if (schemaCache.get(avroSchema) != null
|| fields.stream().anyMatch(field -> avroSchema.getField(field) != null)) {
org.apache.avro.Schema modified = schemaCache.computeIfAbsent(
avroSchema, schema -> org.apache.avro.Schema.createRecord(
avroSchema.getName(), avroSchema.getDoc(), avroSchema.getNamespace(), avroSchema.isError(),
avroSchema.getFields()
.stream()
.filter(f -> !fields.contains(f.name()))
.map(f -> new org.apache.avro.Schema.Field(f.name(), f.schema(), f.doc(),
f.defaultVal(),
f.order()))
.collect(Collectors.toList()))
);

GenericRecord newRecord = new GenericData.Record(modified);
for (org.apache.avro.Schema.Field field : modified.getFields()) {
newRecord.put(field.name(), record.get(field.name()));
}
return newRecord;
}
return record;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/**
* 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.functions.transforms;

import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.stream.Collectors;
import lombok.extern.slf4j.Slf4j;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
import org.apache.pulsar.client.api.Schema;
import org.apache.pulsar.common.schema.SchemaType;

@Slf4j
public class MergeKeyValueStep implements TransformStep {

private final Map<org.apache.avro.Schema, Map<org.apache.avro.Schema, org.apache.avro.Schema>> schemaCache =
new ConcurrentHashMap<>();

@Override
public void process(TransformContext transformContext) {
Schema<?> keySchema = transformContext.getKeySchema();
if (keySchema == null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what does it mean ?
are we assuming that the Key is byte[] or String when we are not using KeyValue Schema ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It means that if we don't have a KeyValue, we don't do anything.

return;
}
if (keySchema.getSchemaInfo().getType() == SchemaType.AVRO
&& transformContext.getValueSchema().getSchemaInfo().getType() == SchemaType.AVRO) {
GenericRecord avroKeyRecord = (GenericRecord) transformContext.getKeyObject();
org.apache.avro.Schema avroKeySchema = avroKeyRecord.getSchema();

GenericRecord avroValueRecord = (GenericRecord) transformContext.getValueObject();
org.apache.avro.Schema avroValueSchema = avroValueRecord.getSchema();

List<String> valueSchemaFieldNames = avroValueSchema.getFields().stream()
.map(org.apache.avro.Schema.Field::name)
.collect(Collectors.toList());
List<org.apache.avro.Schema.Field> fields =
avroKeySchema.getFields().stream()
.filter(field -> !valueSchemaFieldNames.contains(field.name()))
.map(f -> new org.apache.avro.Schema.Field(f.name(), f.schema(), f.doc(),
f.defaultVal(),
f.order()))
.collect(Collectors.toList());
fields.addAll(avroValueSchema.getFields().stream()
.map(f -> new org.apache.avro.Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(),
f.order()))
.collect(Collectors.toList()));

Map<org.apache.avro.Schema, org.apache.avro.Schema> schemaCacheKey =
schemaCache.computeIfAbsent(avroKeySchema, s -> new ConcurrentHashMap<>());
org.apache.avro.Schema modified = schemaCacheKey.computeIfAbsent(avroValueSchema, schema ->
org.apache.avro.Schema.createRecord(
avroValueSchema.getName(),
null,
avroValueSchema.getNamespace(),
false,
fields));
GenericRecord newRecord = new GenericData.Record(modified);
for (String fieldName : valueSchemaFieldNames) {
newRecord.put(fieldName, avroValueRecord.get(fieldName));
}
for (org.apache.avro.Schema.Field field : avroKeySchema.getFields()) {
newRecord.put(field.name(), avroKeyRecord.get(field.name()));
}
transformContext.setValueObject(newRecord);
transformContext.setValueModified(true);
}
}
}
Loading