-
Notifications
You must be signed in to change notification settings - Fork 1.3k
Filter out of order feature rows #273
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
Closed
Closed
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
88 changes: 88 additions & 0 deletions
88
ingestion/src/main/java/feast/ingestion/transform/FilterOutdatedFeatureRow.java
This file contains hidden or 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,88 @@ | ||
| package feast.ingestion.transform; | ||
|
|
||
| import com.google.auto.value.AutoValue; | ||
| import feast.storage.RedisProto; | ||
| import feast.types.FeatureRowProto; | ||
| import org.apache.beam.sdk.coders.VarLongCoder; | ||
| import org.apache.beam.sdk.state.*; | ||
| import org.apache.beam.sdk.transforms.DoFn; | ||
| import org.apache.beam.sdk.transforms.MapElements; | ||
| import org.apache.beam.sdk.transforms.PTransform; | ||
| import org.apache.beam.sdk.transforms.ParDo; | ||
| import org.apache.beam.sdk.values.KV; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.apache.beam.sdk.values.TypeDescriptor; | ||
| import org.apache.beam.sdk.values.TypeDescriptors; | ||
| import org.joda.time.Duration; | ||
|
|
||
| @AutoValue | ||
| abstract public class FilterOutdatedFeatureRow | ||
| extends PTransform<PCollection<FeatureRowProto.FeatureRow>, PCollection<FeatureRowProto.FeatureRow>> { | ||
|
|
||
| public abstract Duration getStateExpiryDuration(); | ||
|
|
||
| public static FilterOutdatedFeatureRow.Builder newBuilder() { | ||
| return new AutoValue_FilterOutdatedFeatureRow.Builder(); | ||
| } | ||
|
|
||
| @AutoValue.Builder | ||
| public abstract static class Builder { | ||
| public abstract Builder setStateExpiryDuration(Duration stateExpiryDuration); | ||
| public abstract FilterOutdatedFeatureRow build(); | ||
| } | ||
|
|
||
| private RedisProto.RedisKey getKey(FeatureRowProto.FeatureRow featureRow) { | ||
| RedisProto.RedisKey.Builder builder = RedisProto.RedisKey.newBuilder().setFeatureSet(featureRow.getFeatureSet()); | ||
| featureRow.getFieldsList().forEach(builder::addEntities); | ||
| return builder.build(); | ||
| } | ||
|
|
||
| public static class FilterDoFn extends DoFn<KV<RedisProto.RedisKey, FeatureRowProto.FeatureRow>, FeatureRowProto.FeatureRow> { | ||
| private static final String LAST_UPDATED = "last_updated"; | ||
| private static final String EXPIRY_TIMER = "state_expiry_timer"; | ||
|
|
||
| private final Duration stateExpiryDuration; | ||
|
|
||
| FilterDoFn(Duration stateExpiryDuration) { | ||
| this.stateExpiryDuration = stateExpiryDuration; | ||
| } | ||
|
|
||
| @StateId(LAST_UPDATED) | ||
| private final StateSpec<ValueState<Long>> lastUpdatedStateSpec = StateSpecs.value(VarLongCoder.of()); | ||
|
|
||
| @TimerId(EXPIRY_TIMER) | ||
| private final TimerSpec stateExpiryTimerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); | ||
|
|
||
|
|
||
| @OnTimer(EXPIRY_TIMER) | ||
| public void onExpiry(OnTimerContext context, @StateId("last_updated") ValueState<Long> lastUpdatedState) { | ||
| lastUpdatedState.clear(); | ||
| } | ||
|
|
||
| @ProcessElement | ||
| public void processElement(ProcessContext context, | ||
| @StateId("last_updated") ValueState<Long> lastUpdatedState, | ||
| @TimerId("state_expiry_timer") Timer stateExpiryTimer) { | ||
| Long lastUpdatedTimestamp = lastUpdatedState.read(); | ||
| Long currentTimestamp = context.element().getValue().getEventTimestamp().getSeconds(); | ||
| if(lastUpdatedTimestamp == null || currentTimestamp > lastUpdatedTimestamp) { | ||
| lastUpdatedState.write(currentTimestamp); | ||
| context.output(context.element().getValue()); | ||
| stateExpiryTimer.offset(stateExpiryDuration).setRelative(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| public PCollection<FeatureRowProto.FeatureRow> expand(PCollection<FeatureRowProto.FeatureRow> featureRowCollection) { | ||
| return featureRowCollection | ||
| .apply( | ||
| MapElements.into( | ||
| TypeDescriptors.kvs(TypeDescriptor.of(RedisProto.RedisKey.class), | ||
| TypeDescriptor.of(FeatureRowProto.FeatureRow.class)) | ||
| ).via((FeatureRowProto.FeatureRow featureRow) -> KV.of(getKey(featureRow), featureRow))) | ||
| .apply(ParDo.of(new FilterDoFn(getStateExpiryDuration()))); | ||
| } | ||
|
|
||
|
|
||
| } | ||
56 changes: 56 additions & 0 deletions
56
ingestion/src/test/java/feast/ingestion/transform/FilterOutdatedFeatureRowTest.java
This file contains hidden or 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 @@ | ||
| package feast.ingestion.transform; | ||
|
|
||
| import com.google.protobuf.Timestamp; | ||
| import feast.types.FeatureRowProto; | ||
| import feast.types.FieldProto; | ||
| import feast.types.ValueProto; | ||
| import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; | ||
| import org.apache.beam.sdk.testing.PAssert; | ||
| import org.apache.beam.sdk.testing.TestPipeline; | ||
| import org.apache.beam.sdk.testing.TestStream; | ||
| import org.apache.beam.sdk.values.PCollection; | ||
| import org.joda.time.Duration; | ||
| import org.joda.time.Instant; | ||
| import org.junit.Rule; | ||
| import org.junit.Test; | ||
|
|
||
| public class FilterOutdatedFeatureRowTest { | ||
|
|
||
| @Rule | ||
| public transient TestPipeline p = TestPipeline.create(); | ||
|
|
||
| private FeatureRowProto.FeatureRow newFeatureRow(String featureSet, String fieldName, Integer value, Long secondsSinceEpoch) { | ||
| return FeatureRowProto.FeatureRow.newBuilder() | ||
| .setEventTimestamp(Timestamp.newBuilder().setSeconds(secondsSinceEpoch).build()) | ||
| .setFeatureSet(featureSet) | ||
| .addFields( | ||
| FieldProto.Field.newBuilder() | ||
| .setName(fieldName) | ||
| .setValue(ValueProto.Value.newBuilder().setInt32Val(value).build())) | ||
| .build(); | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldFilterOutdatedFeatureRow() { | ||
| Duration expiryTime = Duration.standardSeconds(120); | ||
|
|
||
| FeatureRowProto.FeatureRow feature1Recent = newFeatureRow("fs1", "fn", 1, 90L); | ||
| FeatureRowProto.FeatureRow feature2Recent = newFeatureRow("fs2", "fn", 1, 80L); | ||
| FeatureRowProto.FeatureRow feature1Outdated = newFeatureRow("fs1", "fn", 1, 80L); | ||
| FeatureRowProto.FeatureRow feature1ResentAfterExpiry = newFeatureRow("fs1", "fn", 1, 85L); | ||
|
|
||
| TestStream<FeatureRowProto.FeatureRow> featureRowTestStream = TestStream.create(ProtoCoder.of(FeatureRowProto.FeatureRow.class)) | ||
| .advanceWatermarkTo(new Instant(0L)) | ||
| .addElements(feature1Recent, feature2Recent, feature1Outdated) | ||
| .advanceWatermarkTo(new Instant(0L).plus(expiryTime.plus(1))) | ||
| .addElements(feature1ResentAfterExpiry) | ||
| .advanceWatermarkToInfinity(); | ||
|
|
||
| PCollection<FeatureRowProto.FeatureRow> filtered = p.apply(featureRowTestStream) | ||
| .apply(FilterOutdatedFeatureRow.newBuilder().setStateExpiryDuration(expiryTime).build()); | ||
| PAssert.that(filtered).containsInAnyOrder(feature1Recent, feature2Recent, feature1ResentAfterExpiry); | ||
| p.run(); | ||
|
|
||
| } | ||
|
|
||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we make this
>=so that it is possible to overwrite values with the same event_time?