diff --git a/ingestion/src/main/java/feast/ingestion/transform/fn/FilterFeatureRowDoFn.java b/ingestion/src/main/java/feast/ingestion/transform/fn/FilterFeatureRowDoFn.java new file mode 100644 index 0000000000..b5ec7629d0 --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/transform/fn/FilterFeatureRowDoFn.java @@ -0,0 +1,47 @@ +/* + * Copyright 2018 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.transform.fn; + +import feast.types.FeatureProto.Feature; +import feast.types.FeatureRowProto.FeatureRow; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.beam.sdk.transforms.DoFn; + +/** + * Filter FeatureRow to only contain feature with given IDs + */ +public class FilterFeatureRowDoFn extends DoFn { + private final Set featureIds; + + public FilterFeatureRowDoFn(List featureIds) { + this.featureIds = new HashSet<>(featureIds); + } + + @ProcessElement + public void processElement(ProcessContext context) { + FeatureRow input = context.element(); + FeatureRow.Builder output = FeatureRow.newBuilder(input).clearFeatures(); + for (Feature feature : input.getFeaturesList()) { + if (featureIds.contains(feature.getId())) { + output.addFeatures(feature); + } + } + context.output(output.build()); + } +} diff --git a/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java b/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java index c163088716..d6d5e36ad0 100644 --- a/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java +++ b/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java @@ -20,12 +20,15 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.auto.service.AutoService; +import com.google.common.base.Strings; +import feast.ingestion.transform.fn.FilterFeatureRowDoFn; import feast.options.Options; import feast.options.OptionsParser; import feast.source.FeatureSource; import feast.source.FeatureSourceFactory; import feast.source.kafka.deserializer.FeatureRowDeserializer; import feast.source.kafka.deserializer.FeatureRowKeyDeserializer; +import feast.specs.ImportSpecProto.Field; import feast.specs.ImportSpecProto.ImportSpec; import feast.types.FeatureRowProto.FeatureRow; import feast.types.FeatureRowProto.FeatureRowKey; @@ -51,7 +54,6 @@ public class KafkaFeatureSource extends FeatureSource { public static final String KAFKA_FEATURE_SOURCE_TYPE = "kafka"; private ImportSpec importSpec; - @Override public PCollection expand(PInput input) { checkArgument(importSpec.getType().equals(KAFKA_FEATURE_SOURCE_TYPE)); @@ -71,24 +73,33 @@ public PCollection expand(PInput input) { PCollection> featureRowRecord = input.getPipeline().apply(kafkaIOReader); - return - featureRowRecord.apply( - ParDo.of( - new DoFn, FeatureRow>() { - @ProcessElement - public void processElement(ProcessContext processContext) { - KafkaRecord record = processContext.element(); - processContext.output(record.getKV().getValue()); - } - })); + PCollection featureRow = featureRowRecord.apply( + ParDo.of( + new DoFn, FeatureRow>() { + @ProcessElement + public void processElement(ProcessContext processContext) { + KafkaRecord record = processContext.element(); + processContext.output(record.getKV().getValue()); + } + })); + + if (options.discardUnknownFeatures) { + List featureIds = new ArrayList<>(); + for(Field field: importSpec.getSchema().getFieldsList()) { + String featureId = field.getFeatureId(); + if (!Strings.isNullOrEmpty(featureId)) { + featureIds.add(featureId); + } + } + return featureRow.apply(ParDo.of(new FilterFeatureRowDoFn(featureIds))); + } + return featureRow; } public static class KafkaReadOptions implements Options { - - @NotEmpty - public String server; - @NotEmpty - public String topics; + @NotEmpty public String server; + @NotEmpty public String topics; + public boolean discardUnknownFeatures = false; } @AutoService(FeatureSourceFactory.class) diff --git a/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java b/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java index 209ba2b5ea..925094828c 100644 --- a/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java +++ b/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java @@ -20,20 +20,24 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.auto.service.AutoService; +import com.google.common.base.Strings; +import feast.ingestion.transform.fn.FilterFeatureRowDoFn; import feast.options.Options; import feast.options.OptionsParser; import feast.source.FeatureSource; import feast.source.FeatureSourceFactory; +import feast.specs.ImportSpecProto.Field; import feast.specs.ImportSpecProto.ImportSpec; import feast.types.FeatureRowProto.FeatureRow; +import java.util.ArrayList; +import java.util.List; import javax.validation.constraints.AssertTrue; import lombok.Builder; import lombok.NonNull; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; -import org.apache.hadoop.hbase.util.Strings; - /** * Transform for reading {@link feast.types.FeatureRowProto.FeatureRow FeatureRow} proto messages @@ -50,8 +54,7 @@ public class PubSubFeatureSource extends FeatureSource { public static final String PUBSUB_FEATURE_SOURCE_TYPE = "pubsub"; - @NonNull - private ImportSpec importSpec; + @NonNull private ImportSpec importSpec; @Override public PCollection expand(PInput input) { @@ -61,12 +64,24 @@ public PCollection expand(PInput input) { PubsubIO.Read read = readProtos(); - if (!Strings.isEmpty(options.subscription)) { + if (!Strings.isNullOrEmpty(options.subscription)) { read = read.fromSubscription(options.subscription); - } else if (!Strings.isEmpty(options.topic)) { + } else if (!Strings.isNullOrEmpty(options.topic)) { read = read.fromTopic(options.topic); } - return input.getPipeline().apply(read); + PCollection featureRow = input.getPipeline().apply(read); + + if (options.discardUnknownFeatures) { + List featureIds = new ArrayList<>(); + for(Field field: importSpec.getSchema().getFieldsList()) { + String featureId = field.getFeatureId(); + if (!Strings.isNullOrEmpty(featureId)) { + featureIds.add(featureId); + } + } + return featureRow.apply(ParDo.of(new FilterFeatureRowDoFn(featureIds))); + } + return featureRow; } PubsubIO.Read readProtos() { @@ -74,19 +89,18 @@ PubsubIO.Read readProtos() { } public static class PubSubReadOptions implements Options { - public String subscription; public String topic; @AssertTrue(message = "subscription or topic must be set") boolean isValid() { - return !Strings.isEmpty(subscription) || !Strings.isEmpty(topic); + return !Strings.isNullOrEmpty(subscription) || !Strings.isNullOrEmpty(topic); } + public boolean discardUnknownFeatures = false; } @AutoService(FeatureSourceFactory.class) public static class Factory implements FeatureSourceFactory { - @Override public String getType() { return PUBSUB_FEATURE_SOURCE_TYPE; diff --git a/ingestion/src/test/java/feast/ingestion/transform/fn/FilterFeatureRowDoFnTest.java b/ingestion/src/test/java/feast/ingestion/transform/fn/FilterFeatureRowDoFnTest.java new file mode 100644 index 0000000000..0068748969 --- /dev/null +++ b/ingestion/src/test/java/feast/ingestion/transform/fn/FilterFeatureRowDoFnTest.java @@ -0,0 +1,76 @@ +/* + * Copyright 2018 The Feast Authors + * + * Licensed 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 + * + * https://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 feast.ingestion.transform.fn; + +import feast.types.FeatureProto.Feature; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.ValueProto.Value; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; + +public class FilterFeatureRowDoFnTest { + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void shouldIgnoreUnspecifiedFeatureID() { + String featureId1 = "testentity.none.feature1"; + String featureId2 = "testentity.hour.feature2"; + String featureId3 = "testentity.day.feature3"; + + List specifiedFeatureIds = Arrays.asList(featureId1, featureId2, featureId3); + FilterFeatureRowDoFn doFn = new FilterFeatureRowDoFn(specifiedFeatureIds); + + FeatureRow row = + FeatureRow.newBuilder() + .setEntityKey("1234") + .setEntityName("testentity") + .addFeatures( + Feature.newBuilder().setId(featureId1).setValue(Value.newBuilder().setInt64Val(10))) + .addFeatures( + Feature.newBuilder().setId(featureId2).setValue(Value.newBuilder().setInt64Val(11))) + .addFeatures( + Feature.newBuilder().setId(featureId3).setValue(Value.newBuilder().setInt64Val(12))) + // this feature should be ignored + .addFeatures(Feature.newBuilder().setId("testEntity.none.unknown_feature")) + .build(); + + PCollection output = testPipeline.apply(Create.of(row)) + .apply(ParDo.of(doFn)); + + FeatureRow expRow = + FeatureRow.newBuilder() + .setEntityKey("1234") + .setEntityName("testentity") + .addFeatures( + Feature.newBuilder().setId(featureId1).setValue(Value.newBuilder().setInt64Val(10))) + .addFeatures( + Feature.newBuilder().setId(featureId2).setValue(Value.newBuilder().setInt64Val(11))) + .addFeatures( + Feature.newBuilder().setId(featureId3).setValue(Value.newBuilder().setInt64Val(12))) + .build(); + PAssert.that(output).containsInAnyOrder(expRow); + + testPipeline.run(); + } +}