From 24009271288ccd1433d1d125b2f757758517b887 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Thu, 17 Jan 2019 17:21:59 +0800 Subject: [PATCH 01/15] ptransform to coalesce feature rows in batch and streaming --- .../transform/CoalesceFeatureRows.java | 177 ++++++++ .../src/test/java/feast/FeastMatchers.java | 2 +- .../test/java/feast/NormalizeFeatureRows.java | 50 +++ .../transform/CoalesceFeatureRowsTest.java | 405 ++++++++++++++++++ ingestion/src/test/resources/logback-test.xml | 28 ++ 5 files changed, 661 insertions(+), 1 deletion(-) create mode 100644 ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java create mode 100644 ingestion/src/test/java/feast/NormalizeFeatureRows.java create mode 100644 ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java create mode 100644 ingestion/src/test/resources/logback-test.xml diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java new file mode 100644 index 00000000000..a1223a13499 --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java @@ -0,0 +1,177 @@ +package feast.ingestion.transform; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.protobuf.Timestamp; +import feast.types.FeatureProto.Feature; +import feast.types.FeatureRowProto.FeatureRow; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; +import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.joda.time.Duration; +import org.joda.time.Instant; + +public class CoalesceFeatureRows extends + PTransform, PCollection> { + + private static final Comparator TIMESTAMP_COMPARATOR = Comparator + .comparing(Timestamp::getSeconds) + .thenComparing(Timestamp::getNanos); + private static final SerializableFunction KEY_FUNCTION = (row) -> + row.getEntityName() + "=" + row + .getEntityKey(); + private Duration delay = Duration.standardSeconds(10); + + public static FeatureRow combineFeatureRows(Iterable rows) { + FeatureRow latestRow = null; + Map features = new HashMap<>(); + int rowCount = 0; + for (FeatureRow row : rows) { + rowCount += 1; + if (latestRow == null) { + latestRow = row; + } else { + if (TIMESTAMP_COMPARATOR.compare(latestRow.getEventTimestamp(), row.getEventTimestamp()) + < 0) { + // row has later timestamp than agg. + for (Feature feature : row.getFeaturesList()) { + features.put(feature.getId(), feature); + } + latestRow = row; + } else { + for (Feature feature : row.getFeaturesList()) { + String featureId = feature.getId(); + // only insert an older feature if there was no newer one. + if (!features.containsKey(featureId)) { + features.put(featureId, feature); + } + } + } + } + } + + Preconditions.checkNotNull(latestRow); + if (rowCount == 1) { + return latestRow; + } else { + for (Feature feature : latestRow.getFeaturesList()) { + features.put(feature.getId(), feature); + } + return latestRow.toBuilder().clearFeatures().addAllFeatures(features.values()).build(); + } + } + + CoalesceFeatureRows withDelay(Duration delay) { + this.delay = delay; + return this; + } + + @Override + public PCollection expand(PCollection input) { + + PCollection> kvs = input + .apply(WithKeys.of(KEY_FUNCTION).withKeyType(TypeDescriptors.strings())) + .setCoder(KvCoder.of(StringUtf8Coder.of(), ProtoCoder.of(FeatureRow.class))); + + if (kvs.isBounded().equals(IsBounded.UNBOUNDED)) { + return kvs.apply("Configure window", Window.>configure() + .withAllowedLateness(Duration.ZERO) + .discardingFiredPanes() + .triggering(AfterProcessingTime.pastFirstElementInPane())) + .apply(ParDo.of(new CombineStateDoFn(delay))) + .apply(Values.create()); + } else { + return kvs.apply(Combine.perKey(CoalesceFeatureRows::combineFeatureRows)) + .apply(Values.create()); + } + } + + @Slf4j + @AllArgsConstructor + public static class CombineStateDoFn extends + DoFn, KV> { + + @StateId("lastKnownValue") + private final StateSpec> lastKnownValue = + StateSpecs.value(ProtoCoder.of(FeatureRow.class)); + @StateId("newElementsBag") + private final StateSpec> newElementsBag = + StateSpecs.bag(ProtoCoder.of(FeatureRow.class)); + @StateId("lastTimerTimestamp") + private final StateSpec> lastTimerTimestamp = StateSpecs.value(); + @TimerId("bufferTimer") + private final TimerSpec bufferTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + private Duration delay; + + @ProcessElement + public void processElement(ProcessContext context, + @StateId("newElementsBag") BagState newElementsBag, + @TimerId("bufferTimer") Timer bufferTimer, + @StateId("lastTimerTimestamp") ValueState lastTimerTimestampValue) { + newElementsBag.add(context.element().getValue()); + log.debug("Adding FeatureRow to state bag {}", context.element()); + + Instant lastTimerTimestamp = lastTimerTimestampValue.read(); + Instant contextTimestamp = context.timestamp(); + if (lastTimerTimestamp == null || lastTimerTimestamp.isBefore(contextTimestamp) || lastTimerTimestamp.equals(contextTimestamp)) { + lastTimerTimestamp = context.timestamp().plus(delay); + log.debug("Setting timer for key {} to {}", context.element().getKey(), lastTimerTimestamp); + lastTimerTimestampValue.write(lastTimerTimestamp); + bufferTimer.offset(delay).setRelative(); + } + } + + @OnTimer("bufferTimer") + public void bufferOnTimer( + OnTimerContext context, OutputReceiver> out, + @StateId("newElementsBag") BagState newElementsBag, + @StateId("lastKnownValue") ValueState lastKnownValue) { + log.debug("onTimer triggered {}", context.timestamp()); + Iterable rows = newElementsBag.read(); + if (!rows.iterator().hasNext()) { + log.debug("Flush with no new elements"); + return; + } + FeatureRow lastKnown = lastKnownValue.read(); + if (lastKnown != null) { + rows = Iterables.concat(Collections.singleton(lastKnown), newElementsBag.read()); + } + // Check if we have more than one value in our list. + FeatureRow row = combineFeatureRows(rows); + log.debug("Timer fired and added FeatureRow to output {}", row); + // Clear the elements now that they have been processed + newElementsBag.clear(); + lastKnownValue.write(row); + + // Output the value stored in the the processed que which matches this timers time + out.output(KV.of(KEY_FUNCTION.apply(row), row)); + } + } +} diff --git a/ingestion/src/test/java/feast/FeastMatchers.java b/ingestion/src/test/java/feast/FeastMatchers.java index c0a6a8a7dbc..eb469b5c934 100644 --- a/ingestion/src/test/java/feast/FeastMatchers.java +++ b/ingestion/src/test/java/feast/FeastMatchers.java @@ -26,7 +26,7 @@ public class FeastMatchers { /** * Can be used with the PAssert.that(..).satisfies(fn) method. */ - public static SerializableFunction, Void> hasCount(int count) { + public static SerializableFunction, Void> hasCount(long count) { return (Iterable iterable) -> { Assert.assertEquals(count, Lists.newArrayList(iterable).size()); return null; diff --git a/ingestion/src/test/java/feast/NormalizeFeatureRows.java b/ingestion/src/test/java/feast/NormalizeFeatureRows.java new file mode 100644 index 00000000000..626f8800431 --- /dev/null +++ b/ingestion/src/test/java/feast/NormalizeFeatureRows.java @@ -0,0 +1,50 @@ +/* + * 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; + +import com.google.common.collect.Lists; +import com.google.common.primitives.UnsignedBytes; +import feast.types.FeatureProto.Feature; +import feast.types.FeatureRowExtendedProto.FeatureRowExtended; +import feast.types.FeatureRowProto.FeatureRow; +import java.util.List; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; + +public class NormalizeFeatureRows + extends PTransform, PCollection> { + + public static FeatureRow orderedFeatureRow(FeatureRow row) { + List features = Lists.newArrayList(row.getFeaturesList()); + features.sort( + (f1, f2) -> + UnsignedBytes.lexicographicalComparator().compare(f1.toByteArray(), f2.toByteArray())); + return row.toBuilder().clearFeatures().addAllFeatures(features).build(); + } + + @Override + public PCollection expand(PCollection input) { + return input + .apply( + "normalize rows", + MapElements.into(TypeDescriptor.of(FeatureRow.class)).via( + NormalizeFeatureRows::orderedFeatureRow)); + } +} diff --git a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java new file mode 100644 index 00000000000..bc0b91cefcb --- /dev/null +++ b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java @@ -0,0 +1,405 @@ +/* + * 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; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +import com.google.common.collect.Lists; +import com.google.protobuf.Timestamp; +import feast.FeastMatchers; +import feast.NormalizeFeatureRows; +import feast.ingestion.model.Values; +import feast.types.FeatureProto.Feature; +import feast.types.FeatureRowProto.FeatureRow; +import java.util.List; +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.transforms.Count; +import org.apache.beam.sdk.transforms.Create; +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 CoalesceFeatureRowsTest { + + @Rule + public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void testBatch_withDistictKeys_shouldPassThroughNonIntersectingKeys() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1").build(), + FeatureRow.newBuilder().setEntityKey("2").build()); + + PCollection input = pipeline.apply(Create.of(rows)) + .setCoder(ProtoCoder.of(FeatureRow.class)); + + PCollection output = input.apply(new CoalesceFeatureRows()); + + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(2L); + PAssert.that(output).containsInAnyOrder(rows); + + pipeline.run(); + } + + @Test + public void test_withNoFeaturesSameTimestamp_shouldReturn1() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1").build(), + FeatureRow.newBuilder().setEntityKey("1").build()); + PCollection input = pipeline.apply(Create.of(rows)) + .setCoder(ProtoCoder.of(FeatureRow.class)); + + PCollection output = input.apply(new CoalesceFeatureRows()); + + assertThat(CoalesceFeatureRows.combineFeatureRows(rows), equalTo(rows.get(0))); + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); + PAssert.that(output).containsInAnyOrder(rows.get(0)); + + pipeline.run(); + } + + @Test + public void testBatch_withNoFeaturesDifferentSeconds_shouldReturnLatest() { + List rows1 = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setSeconds(1)).build(), + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)).build()); + + assertThat(CoalesceFeatureRows.combineFeatureRows(rows1), equalTo(rows1.get(1))); + assertThat(CoalesceFeatureRows.combineFeatureRows(Lists.reverse(rows1)), + equalTo(rows1.get(1))); + } + + + @Test + public void testBatch_withNoFeaturesDifferentNanos_shouldReturnLatest() { + List rows1 = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setNanos(1)).build(), + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setNanos(2)).build()); + + assertThat(CoalesceFeatureRows.combineFeatureRows(rows1), equalTo(rows1.get(1))); + assertThat(CoalesceFeatureRows.combineFeatureRows(Lists.reverse(rows1)), + equalTo(rows1.get(1))); + } + + @Test + public void testBatch_shouldMergeFeatures() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build()); + PCollection input = pipeline.apply(Create.of(rows)) + .setCoder(ProtoCoder.of(FeatureRow.class)); + + PCollection output = input.apply(new CoalesceFeatureRows()); + + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + + pipeline.run(); + } + + @Test + public void testStream_shouldMergeFeatures() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build()); + + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkTo(start) + .addElements(rows.get(0)) + .addElements(rows.get(1)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); + PAssert.that(output).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + pipeline.run(); + } + + @Test + public void testStream_shouldMergeFeatures_emittingPanes() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build()); + + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkTo(start) + .addElements(rows.get(0)) + .advanceWatermarkTo(start.plus(delay).plus(delay.dividedBy(2))) + .addElements(rows.get(1)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input + .apply(new CoalesceFeatureRows().withDelay(delay)); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build() + ); + pipeline.run(); + } + + @Test + public void testStream_shouldIncludeRowOnTimerEdge() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) + .build()); + + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkTo(start) + .addElements(rows.get(0)) + .advanceWatermarkTo(start.plus(delay)) // Previous timer triggers + .addElements(rows.get(1)) + .advanceWatermarkTo(start.plus(delay).plus(delay).plus(delay)) + .addElements(rows.get(2)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input + .apply(new CoalesceFeatureRows().withDelay(delay)); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) + .build() + ); + pipeline.run(); + } + + @Test + public void testStream_shouldMergeFeatures_emittingPanes_overlappingTimers() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkTo(start) + .addElements(rows.get(0)) + .advanceWatermarkTo(start.plus(delay.dividedBy(2))) + // second event before time triggers + .addElements(rows.get(1)) + .advanceWatermarkTo(start.plus(delay)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + pipeline.run(); + } + + @Test + public void testStream_shouldNotSetTimerWhilePending() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) + .build() + ); + + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkTo(start) + .addElements(rows.get(0)) + .advanceWatermarkTo(start.plus(delay.dividedBy(2))) + .addElements(rows.get(1)) // this should not reset the timer as the first is still pending. + // timer should trigger causing the first output row + .advanceWatermarkTo(start.plus(delay).plus(delay.dividedBy(2))) + .addElements(rows.get(2)) // this should cause a second output row. + .advanceWatermarkTo(start.plus(delay).plus(delay)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) + .build() + ); + pipeline.run(); + } + + + @Test + public void test_shouldPickLatestFeatures() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp( + Timestamp.newBuilder().setSeconds(1)) // old row with non unique feature + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(2))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setSeconds(1)) // old row with unique feature + .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) + .build()); + + assertThat(CoalesceFeatureRows.combineFeatureRows(rows), equalTo( + FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(2))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) + .build() + )); + } + + @Test + public void testStream_withNoDelay_shouldNotTriggerBufferTimer() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + + Duration delay = Duration.standardSeconds(0); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .addElements(rows.get(0)) + // Even though we have no delay, we should still get a single output row as processing time has not advanced + .addElements(rows.get(1)) + .advanceProcessingTime(delay.plus(1)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + pipeline.run(); + } + + @Test + public void testStream_withNoInput() { + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows()); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(0L)); + pipeline.run(); + } + + @Test + public void testBatch_withNoInput() { + PCollection input = pipeline.apply(Create.empty(ProtoCoder.of(FeatureRow.class))); + PCollection output = input.apply(new CoalesceFeatureRows()); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(0L)); + pipeline.run(); + } +} \ No newline at end of file diff --git a/ingestion/src/test/resources/logback-test.xml b/ingestion/src/test/resources/logback-test.xml new file mode 100644 index 00000000000..9a301f3bc63 --- /dev/null +++ b/ingestion/src/test/resources/logback-test.xml @@ -0,0 +1,28 @@ + + + + + + + %d{HH:mm:ss} [%thread] %-5level %logger{50} - %msg%n + + + + + + \ No newline at end of file From 262cd0990a9ea5379180c0c559fa7dffb9d2a680 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Fri, 18 Jan 2019 18:11:24 +0800 Subject: [PATCH 02/15] add coalesce feature rows transform, to build state of latest row per key in the global window --- .../main/java/feast/ingestion/ImportJob.java | 25 ++-- .../feast/ingestion/boot/ImportJobModule.java | 10 +- .../ingestion/config/ImportSpecSupplier.java | 6 +- .../metrics/FeastMetricsPubSubSink.java | 4 +- ...ons.java => ImportJobPipelineOptions.java} | 11 +- .../feast/ingestion/options/JobOptions.java | 66 +++++++++++ .../transform/CoalesceFeatureRows.java | 112 ++++++++++++++---- .../transform/CoalescePFeatureRows.java | 47 ++++++++ .../transform/ErrorsStoreTransform.java | 4 +- .../feast/ingestion/values/PFeatureRows.java | 1 - ingestion/src/test/java/feast/TestHelper.java | 3 - .../feast/ingestion/ImportJobCSVTest.java | 12 +- .../config/ImportSpecSupplierTest.java | 7 +- .../ingestion/options/JobOptionsTest.java | 89 ++++++++++++++ .../transform/CoalesceFeatureRowsTest.java | 86 ++++++++++---- .../transform/ErrorsStoreTransformTest.java | 6 +- protos/feast/specs/ImportSpec.proto | 15 ++- 17 files changed, 409 insertions(+), 95 deletions(-) rename ingestion/src/main/java/feast/ingestion/options/{ImportJobOptions.java => ImportJobPipelineOptions.java} (90%) create mode 100644 ingestion/src/main/java/feast/ingestion/options/JobOptions.java create mode 100644 ingestion/src/main/java/feast/ingestion/transform/CoalescePFeatureRows.java create mode 100644 ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java diff --git a/ingestion/src/main/java/feast/ingestion/ImportJob.java b/ingestion/src/main/java/feast/ingestion/ImportJob.java index f048e5a5854..f757a6f4471 100644 --- a/ingestion/src/main/java/feast/ingestion/ImportJob.java +++ b/ingestion/src/main/java/feast/ingestion/ImportJob.java @@ -28,7 +28,9 @@ import feast.ingestion.boot.PipelineModule; import feast.ingestion.config.ImportSpecSupplier; import feast.ingestion.model.Specs; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; +import feast.ingestion.options.JobOptions; +import feast.ingestion.transform.CoalescePFeatureRows; import feast.ingestion.transform.ErrorsStoreTransform; import feast.ingestion.transform.ReadFeaturesTransform; import feast.ingestion.transform.ServingStoreTransform; @@ -39,6 +41,7 @@ import feast.ingestion.transform.fn.LoggerDoFn; import feast.ingestion.transform.fn.RoundEventTimestampsDoFn; import feast.ingestion.values.PFeatureRows; +import feast.options.OptionsParser; import feast.specs.ImportSpecProto.ImportSpec; import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; @@ -82,7 +85,7 @@ public class ImportJob { private final WarehouseStoreTransform warehouseStoreTransform; private final ErrorsStoreTransform errorsStoreTransform; private final boolean dryRun; - private final ImportJobOptions options; + private final ImportJobPipelineOptions options; private final Specs specs; @Inject @@ -93,7 +96,7 @@ public ImportJob( ServingStoreTransform servingStoreTransform, WarehouseStoreTransform warehouseStoreTransform, ErrorsStoreTransform errorsStoreTransform, - ImportJobOptions options, + ImportJobPipelineOptions options, Specs specs) { this.pipeline = pipeline; this.importSpec = importSpec; @@ -112,8 +115,8 @@ public static void main(String[] args) { public static PipelineResult mainWithResult(String[] args) { log.info("Arguments: " + Arrays.toString(args)); - ImportJobOptions options = - PipelineOptionsFactory.fromArgs(args).withValidation().as(ImportJobOptions.class); + ImportJobPipelineOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(ImportJobPipelineOptions.class); if (options.getJobName().isEmpty()) { options.setJobName(generateName()); } @@ -142,6 +145,8 @@ public void expand() { TypeDescriptor.of(FeatureRowExtended.class), ProtoCoder.of(FeatureRowExtended.class)); coderRegistry.registerCoderForType(TypeDescriptor.of(TableRow.class), TableRowJsonCoder.of()); + JobOptions jobOptions = OptionsParser.parse(importSpec.getJobOptionsMap(), JobOptions.class); + try { log.info(JsonFormat.printer().print(importSpec)); } catch (InvalidProtocolBufferException e) { @@ -151,8 +156,8 @@ public void expand() { specs.validate(); PCollection features = pipeline.apply("Read", readFeaturesTransform); - if (options.getLimit() != null && options.getLimit() > 0) { - features = features.apply(Sample.any(options.getLimit())); + if (jobOptions.getSampleLimit() > 0) { + features = features.apply(Sample.any(jobOptions.getSampleLimit())); } PCollection featuresExtended = @@ -170,6 +175,12 @@ public void expand() { ParDo.of(new RoundEventTimestampsDoFn())), pFeatureRows.getErrors()); + if (jobOptions.isCoalesceRowsEnabled()) { + pFeatureRows = pFeatureRows.apply("foo", new CoalescePFeatureRows( + jobOptions.getCoalesceRowsDelaySeconds(), + jobOptions.getCoalesceRowsTimeoutSeconds())); + } + if (!dryRun) { List> errors = Lists.newArrayList(); pFeatureRows = pFeatureRows.apply("Write to Serving Stores", servingStoreTransform); diff --git a/ingestion/src/main/java/feast/ingestion/boot/ImportJobModule.java b/ingestion/src/main/java/feast/ingestion/boot/ImportJobModule.java index ce3bb622072..7e9e235b5bc 100644 --- a/ingestion/src/main/java/feast/ingestion/boot/ImportJobModule.java +++ b/ingestion/src/main/java/feast/ingestion/boot/ImportJobModule.java @@ -21,7 +21,7 @@ import com.google.inject.Provides; import com.google.inject.Singleton; import feast.ingestion.model.Specs; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; import feast.ingestion.service.CoreSpecService; import feast.ingestion.service.FileSpecService; import feast.ingestion.service.SpecService; @@ -42,24 +42,24 @@ */ public class ImportJobModule extends AbstractModule { - private final ImportJobOptions options; + private final ImportJobPipelineOptions options; private ImportSpec importSpec; - public ImportJobModule(ImportJobOptions options, ImportSpec importSpec) { + public ImportJobModule(ImportJobPipelineOptions options, ImportSpec importSpec) { this.options = options; this.importSpec = importSpec; } @Override protected void configure() { - bind(ImportJobOptions.class).toInstance(options); + bind(ImportJobPipelineOptions.class).toInstance(options); bind(PipelineOptions.class).toInstance(options); bind(ImportSpec.class).toInstance(importSpec); } @Provides @Singleton - Builder provideSpecService(ImportJobOptions options) { + Builder provideSpecService(ImportJobPipelineOptions options) { if (options.getCoreApiUri() != null) { return new CoreSpecService.Builder(options.getCoreApiUri()); } else if (options.getCoreApiSpecPath() != null) { diff --git a/ingestion/src/main/java/feast/ingestion/config/ImportSpecSupplier.java b/ingestion/src/main/java/feast/ingestion/config/ImportSpecSupplier.java index 04d2e128597..ba1b431bfae 100644 --- a/ingestion/src/main/java/feast/ingestion/config/ImportSpecSupplier.java +++ b/ingestion/src/main/java/feast/ingestion/config/ImportSpecSupplier.java @@ -22,17 +22,17 @@ import java.nio.file.Paths; import java.util.Base64; import java.util.function.Supplier; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; import feast.ingestion.util.ProtoUtil; import feast.specs.ImportSpecProto.ImportSpec; public class ImportSpecSupplier implements Supplier { - private ImportJobOptions options; + private ImportJobPipelineOptions options; private ImportSpec importSpec; @Inject - public ImportSpecSupplier(ImportJobOptions options) { + public ImportSpecSupplier(ImportJobPipelineOptions options) { this.options = options; } diff --git a/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java b/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java index 06b35a93cb7..5bf79e20df0 100644 --- a/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java +++ b/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; import feast.types.FeatureRowProto.FeatureRow; import feast.types.GranularityProto.Granularity; @@ -104,7 +104,7 @@ public interface FeastMetricsPubSubSinkOptions extends PipelineOptions { public static class FeastMetricsPubSubSinkOptionsRegistrar implements PipelineOptionsRegistrar { @Override public Iterable> getPipelineOptions() { - return Collections.singleton(ImportJobOptions.class); + return Collections.singleton(ImportJobPipelineOptions.class); } } } diff --git a/ingestion/src/main/java/feast/ingestion/options/ImportJobOptions.java b/ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java similarity index 90% rename from ingestion/src/main/java/feast/ingestion/options/ImportJobOptions.java rename to ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java index 6da7d0f726f..2ec4fa44798 100644 --- a/ingestion/src/main/java/feast/ingestion/options/ImportJobOptions.java +++ b/ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.options.Validation.Required; -public interface ImportJobOptions extends PipelineOptions { +public interface ImportJobPipelineOptions extends PipelineOptions { @Description("Import spec yaml file path") @Required(groups = {"importSpec"}) @@ -67,11 +67,6 @@ public interface ImportJobOptions extends PipelineOptions { @Override void setMetricsSink(Class metricsSink); - @Description("Filter a to a max number if rows from the read source") - Long getLimit(); - - void setLimit(Long value); - @Description( "Set an errors store type. One of: [stderr, stdout, file.json]. Note that you should not use " + "stderr/stdout in production unless your data volume is extremely small.") @@ -88,11 +83,11 @@ public interface ImportJobOptions extends PipelineOptions { void setErrorsStoreOptions(String value); @AutoService(PipelineOptionsRegistrar.class) - class ImportJobOptionsRegistrar implements PipelineOptionsRegistrar { + class ImportJobPipelineOptionsRegistrar implements PipelineOptionsRegistrar { @Override public Iterable> getPipelineOptions() { - return Collections.singleton(ImportJobOptions.class); + return Collections.singleton(ImportJobPipelineOptions.class); } } } diff --git a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java new file mode 100644 index 00000000000..8cbe9ade7a9 --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java @@ -0,0 +1,66 @@ +/* + * Copyright 2019 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 + * + * 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 feast.ingestion.options; + +import com.fasterxml.jackson.annotation.JsonProperty; +import feast.options.Options; + +public class JobOptions implements Options { + + private long sampleLimit; + private boolean coalesceRowsEnabled; + private long coalesceRowsDelaySeconds; + private long coalesceRowsTimeoutSeconds; + + @JsonProperty(value = "sample.limit") + public long getSampleLimit() { + return sampleLimit; + } + + public void setSampleimit(long sampleLimit) { + this.sampleLimit = sampleLimit; + } + + @JsonProperty(value = "coalesceRows.enabled") + public boolean isCoalesceRowsEnabled() { + return coalesceRowsEnabled; + } + + public void setCoalesceRowsEnabled(boolean coalesceRows) { + this.coalesceRowsEnabled = coalesceRows; + } + + @JsonProperty(value = "coalesceRows.delaySeconds") + public long getCoalesceRowsDelaySeconds() { + return coalesceRowsDelaySeconds; + } + + public void setCoalesceRowsDelaySeconds(long coalesceRowsDelaySeconds) { + this.coalesceRowsDelaySeconds = coalesceRowsDelaySeconds; + } + + @JsonProperty(value = "coalesceRows.timeoutSeconds") + public long getCoalesceRowsTimeoutSeconds() { + return coalesceRowsTimeoutSeconds; + } + + public void setCoalesceRowsTimeoutSeconds(long coalesceRowsTimeoutSeconds) { + this.coalesceRowsTimeoutSeconds = coalesceRowsTimeoutSeconds; + } + +} diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java index a1223a13499..4bc5f172553 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java @@ -1,3 +1,20 @@ +/* + * Copyright 2019 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 + * + * 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 feast.ingestion.transform; import com.google.common.base.Preconditions; @@ -5,6 +22,7 @@ import com.google.protobuf.Timestamp; import feast.types.FeatureProto.Feature; import feast.types.FeatureRowProto.FeatureRow; +import feast.types.FeatureRowProto.FeatureRowKey; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -12,7 +30,6 @@ import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.StateSpec; @@ -34,20 +51,50 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; +/** + * Takes FeatureRow, and merges them if they have the same FeatureRowKey, so that the latest values + * will be emitted. It emits only once for batch. + * + * For streaming we emits after a delay of 10 seconds (event time) by default we keep the previous + * state around for merging with future events. These timeout after 30 minutes by default. + */ public class CoalesceFeatureRows extends PTransform, PCollection> { private static final Comparator TIMESTAMP_COMPARATOR = Comparator .comparing(Timestamp::getSeconds) .thenComparing(Timestamp::getNanos); - private static final SerializableFunction KEY_FUNCTION = (row) -> - row.getEntityName() + "=" + row - .getEntityKey(); - private Duration delay = Duration.standardSeconds(10); + private static final SerializableFunction KEY_FUNCTION = (row) -> + FeatureRowKey.newBuilder() + .setEntityName(row.getEntityName()) + .setEntityKey(row.getEntityKey()) + .setGranularity(row.getGranularity()) + .setEventTimestamp(row.getEventTimestamp()).build(); + + private static final Duration DEFAULT_DELAY = Duration.standardSeconds(10); + private static final Duration DEFAULT_TIMEOUT = Duration.ZERO; + + private Duration delay; + private Duration timeout; + + CoalesceFeatureRows() { + this(0, 0); + + } + + public CoalesceFeatureRows(long delaySeconds, long timeoutSeconds) { + this(Duration.standardSeconds(delaySeconds), Duration.standardSeconds(timeoutSeconds)); + } + + public CoalesceFeatureRows(Duration delay, Duration timeout) { + this.delay = (delay.isEqual(Duration.ZERO)) ? DEFAULT_DELAY : delay; + this.timeout = (timeout.isEqual(Duration.ZERO)) ? DEFAULT_TIMEOUT : timeout; + } + public static FeatureRow combineFeatureRows(Iterable rows) { FeatureRow latestRow = null; @@ -88,24 +135,18 @@ public static FeatureRow combineFeatureRows(Iterable rows) { } } - CoalesceFeatureRows withDelay(Duration delay) { - this.delay = delay; - return this; - } - @Override public PCollection expand(PCollection input) { - - PCollection> kvs = input - .apply(WithKeys.of(KEY_FUNCTION).withKeyType(TypeDescriptors.strings())) - .setCoder(KvCoder.of(StringUtf8Coder.of(), ProtoCoder.of(FeatureRow.class))); + PCollection> kvs = input + .apply(WithKeys.of(KEY_FUNCTION).withKeyType(TypeDescriptor.of(FeatureRowKey.class))) + .setCoder(KvCoder.of(ProtoCoder.of(FeatureRowKey.class), ProtoCoder.of(FeatureRow.class))); if (kvs.isBounded().equals(IsBounded.UNBOUNDED)) { - return kvs.apply("Configure window", Window.>configure() + return kvs.apply("Configure window", Window.>configure() .withAllowedLateness(Duration.ZERO) .discardingFiredPanes() .triggering(AfterProcessingTime.pastFirstElementInPane())) - .apply(ParDo.of(new CombineStateDoFn(delay))) + .apply(ParDo.of(new CombineStateDoFn(delay, timeout))) .apply(Values.create()); } else { return kvs.apply(Combine.perKey(CoalesceFeatureRows::combineFeatureRows)) @@ -116,7 +157,7 @@ public PCollection expand(PCollection input) { @Slf4j @AllArgsConstructor public static class CombineStateDoFn extends - DoFn, KV> { + DoFn, KV> { @StateId("lastKnownValue") private final StateSpec> lastKnownValue = @@ -128,19 +169,29 @@ public static class CombineStateDoFn extends private final StateSpec> lastTimerTimestamp = StateSpecs.value(); @TimerId("bufferTimer") private final TimerSpec bufferTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + @TimerId("timeoutTimer") + private final TimerSpec timeoutTimer = TimerSpecs.timer(TimeDomain.EVENT_TIME); + private Duration delay; + private Duration timeout; @ProcessElement public void processElement(ProcessContext context, @StateId("newElementsBag") BagState newElementsBag, @TimerId("bufferTimer") Timer bufferTimer, + @TimerId("timeoutTimer") Timer timeoutTimer, @StateId("lastTimerTimestamp") ValueState lastTimerTimestampValue) { newElementsBag.add(context.element().getValue()); log.debug("Adding FeatureRow to state bag {}", context.element()); Instant lastTimerTimestamp = lastTimerTimestampValue.read(); Instant contextTimestamp = context.timestamp(); - if (lastTimerTimestamp == null || lastTimerTimestamp.isBefore(contextTimestamp) || lastTimerTimestamp.equals(contextTimestamp)) { + if (lastTimerTimestamp == null && timeout.isLongerThan(Duration.ZERO)) { + // We never timeout the state if a timeout is not set. + timeoutTimer.offset(timeout).setRelative(); + } + if (lastTimerTimestamp == null || lastTimerTimestamp.isBefore(contextTimestamp) + || lastTimerTimestamp.equals(contextTimestamp)) { lastTimerTimestamp = context.timestamp().plus(delay); log.debug("Setting timer for key {} to {}", context.element().getKey(), lastTimerTimestamp); lastTimerTimestampValue.write(lastTimerTimestamp); @@ -150,10 +201,29 @@ public void processElement(ProcessContext context, @OnTimer("bufferTimer") public void bufferOnTimer( - OnTimerContext context, OutputReceiver> out, + OnTimerContext context, OutputReceiver> out, + @StateId("newElementsBag") BagState newElementsBag, + @StateId("lastKnownValue") ValueState lastKnownValue) { + log.debug("bufferOnTimer triggered {}", context.timestamp()); + flush(out, newElementsBag, lastKnownValue); + } + + @OnTimer("timeoutTimer") + public void timeoutOnTimer( + OnTimerContext context, OutputReceiver> out, + @StateId("newElementsBag") BagState newElementsBag, + @StateId("lastKnownValue") ValueState lastKnownValue) { + log.debug("timeoutOnTimer triggered {}", context.timestamp()); + flush(out, newElementsBag, lastKnownValue); + newElementsBag.clear(); + lastKnownValue.clear(); + } + + public void flush( + OutputReceiver> out, @StateId("newElementsBag") BagState newElementsBag, @StateId("lastKnownValue") ValueState lastKnownValue) { - log.debug("onTimer triggered {}", context.timestamp()); + log.debug("Flush triggered"); Iterable rows = newElementsBag.read(); if (!rows.iterator().hasNext()) { log.debug("Flush with no new elements"); diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalescePFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalescePFeatureRows.java new file mode 100644 index 00000000000..cf7932f0155 --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalescePFeatureRows.java @@ -0,0 +1,47 @@ +/* + * Copyright 2019 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 + * + * 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 feast.ingestion.transform; + +import feast.ingestion.values.PFeatureRows; +import feast.types.FeatureRowExtendedProto.FeatureRowExtended; +import feast.types.FeatureRowProto.FeatureRow; +import lombok.AllArgsConstructor; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; + +/** + * This class is a work around to make some refactoring easier. PFeatureRows should be deprecated. + */ +@AllArgsConstructor +public class CoalescePFeatureRows extends + PTransform { + + private long delaySeconds; + private long timeoutSeconds; + + @Override + public PFeatureRows expand(PFeatureRows input) { + PCollection output = input.getMain() + .apply(MapElements.into(TypeDescriptor.of(FeatureRow.class)) + .via(FeatureRowExtended::getRow)) + .apply(new CoalesceFeatureRows(delaySeconds, timeoutSeconds)) + .apply(new ToFeatureRowExtended()); + return PFeatureRows.of(output, input.getErrors()); + } +} diff --git a/ingestion/src/main/java/feast/ingestion/transform/ErrorsStoreTransform.java b/ingestion/src/main/java/feast/ingestion/transform/ErrorsStoreTransform.java index 97799a0ae86..7b8e41030e9 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/ErrorsStoreTransform.java +++ b/ingestion/src/main/java/feast/ingestion/transform/ErrorsStoreTransform.java @@ -22,7 +22,7 @@ import com.google.inject.Inject; import feast.ingestion.model.Specs; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; import feast.ingestion.transform.FeatureIO.Write; import feast.specs.StorageSpecProto.StorageSpec; import feast.storage.ErrorsStore; @@ -44,7 +44,7 @@ public class ErrorsStoreTransform extends FeatureIO.Write { @Inject public ErrorsStoreTransform( - ImportJobOptions options, Specs specs, List errorsStores) { + ImportJobPipelineOptions options, Specs specs, List errorsStores) { this.specs = specs; this.errorsStores = errorsStores; this.errorsStoreType = options.getErrorsStoreType(); diff --git a/ingestion/src/main/java/feast/ingestion/values/PFeatureRows.java b/ingestion/src/main/java/feast/ingestion/values/PFeatureRows.java index 3c46e92f158..86a710ddae6 100644 --- a/ingestion/src/main/java/feast/ingestion/values/PFeatureRows.java +++ b/ingestion/src/main/java/feast/ingestion/values/PFeatureRows.java @@ -14,7 +14,6 @@ * limitations under the License. * */ - package feast.ingestion.values; import feast.ingestion.transform.fn.BaseFeatureDoFn; diff --git a/ingestion/src/test/java/feast/TestHelper.java b/ingestion/src/test/java/feast/TestHelper.java index ce554d4abee..34984266f6f 100644 --- a/ingestion/src/test/java/feast/TestHelper.java +++ b/ingestion/src/test/java/feast/TestHelper.java @@ -23,9 +23,6 @@ import com.google.gson.Gson; import java.io.IOException; import java.io.OutputStream; -import java.util.Optional; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import feast.ingestion.options.ImportJobOptions; public class TestHelper { public static void writeYaml(Object model, OutputStream outputStream) throws IOException { diff --git a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java index c6adf7dd22d..e6741c648a9 100644 --- a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java +++ b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java @@ -34,7 +34,7 @@ import feast.ingestion.boot.TestPipelineModule; import feast.ingestion.model.Features; import feast.ingestion.model.Values; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; import feast.ingestion.service.SpecRetrievalException; import feast.ingestion.util.ProtoUtil; import feast.specs.ImportSpecProto.ImportSpec; @@ -76,9 +76,9 @@ public ImportSpec initImportSpec(ImportSpec importSpec, String dataFile) { return importSpec.toBuilder().putOptions("path", dataFile).build(); } - public ImportJobOptions initOptions() { + public ImportJobPipelineOptions initOptions() { Path path = Paths.get(Resources.getResource("core_specs/").getPath()); - ImportJobOptions options = PipelineOptionsFactory.create().as(ImportJobOptions.class); + ImportJobPipelineOptions options = PipelineOptionsFactory.create().as(ImportJobPipelineOptions.class); options.setCoreApiSpecPath(path.toString()); options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); return options; @@ -108,7 +108,7 @@ public void testImportCSV() throws IOException { Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n2,202,b\n3,303,c\n"); importSpec = initImportSpec(importSpec, csvFile.toString()); - ImportJobOptions options = initOptions(); + ImportJobPipelineOptions options = initOptions(); options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); Injector injector = @@ -199,7 +199,7 @@ public void testImportCSVUnknownServingStoreError() throws IOException { Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n2,202,b\n3,303,c\n"); importSpec = initImportSpec(importSpec, csvFile.toString()); - ImportJobOptions options = initOptions(); + ImportJobPipelineOptions options = initOptions(); Injector injector = Guice.createInjector( @@ -238,7 +238,7 @@ public void testImportWithErrors() throws IOException { Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n2,202,b\n3,303,c\n"); importSpec = initImportSpec(importSpec, csvFile.toString()); - ImportJobOptions options = initOptions(); + ImportJobPipelineOptions options = initOptions(); options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); Injector injector = diff --git a/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java b/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java index a25a2179ca5..cf22ba523c5 100644 --- a/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java +++ b/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java @@ -18,11 +18,8 @@ package feast.ingestion.config; import feast.ingestion.util.DateUtil; -import feast.ingestion.options.ImportJobOptions; -import com.google.protobuf.InvalidProtocolBufferException; +import feast.ingestion.options.ImportJobPipelineOptions; import com.google.protobuf.util.JsonFormat; -import feast.ingestion.options.ImportJobOptions; -import feast.ingestion.util.DateUtil; import feast.specs.ImportSpecProto.Field; import feast.specs.ImportSpecProto.ImportSpec; import feast.specs.ImportSpecProto.Schema; @@ -89,7 +86,7 @@ public void testSupplierImportSpecYamlFile() throws IOException { printWriter.print(importSpecYaml); } - ImportJobOptions options = PipelineOptionsFactory.create().as(ImportJobOptions.class); + ImportJobPipelineOptions options = PipelineOptionsFactory.create().as(ImportJobPipelineOptions.class); options.setImportSpecYamlFile(yamlFile.toString()); ImportSpec importSpec = new ImportSpecSupplier(options).get(); diff --git a/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java b/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java new file mode 100644 index 00000000000..aacc83886a2 --- /dev/null +++ b/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java @@ -0,0 +1,89 @@ +/* + * Copyright 2019 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 + * + * 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 feast.ingestion.options; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import feast.options.OptionsParser; +import java.util.HashMap; +import java.util.Map; +import org.junit.Test; + +public class JobOptionsTest { + + @Test + public void test_shouldParseAll() { + Map map = new HashMap<>(); + map.put("coalesceRows.enabled", "true"); + map.put("coalesceRows.delaySeconds", "123"); + map.put("coalesceRows.timeoutSeconds", "1800"); + map.put("sample.limit", "1234"); + + + JobOptions options = OptionsParser.parse(map, JobOptions.class); + assertEquals(options.getSampleLimit(), 1234L); + assertEquals(options.getCoalesceRowsDelaySeconds(), 123L); + assertTrue(options.isCoalesceRowsEnabled()); + assertEquals(options.getCoalesceRowsTimeoutSeconds(), 1800L); + } + + @Test + public void test_shouldParseEmptyOptions() { + JobOptions options = OptionsParser.parse(new HashMap<>(), JobOptions.class); + assertEquals(options.getSampleLimit(), 0L); + assertEquals(options.getCoalesceRowsDelaySeconds(), 0L); + assertFalse(options.isCoalesceRowsEnabled()); + assertEquals(options.getCoalesceRowsTimeoutSeconds(), 0L); + } + + @Test + public void test_shouldParseSampleLimit() { + Map map = new HashMap<>(); + map.put("sample.limit", "1234"); + JobOptions options = OptionsParser.parse(map, JobOptions.class); + assertEquals(options.getSampleLimit(), 1234L); + } + + @Test + public void test_shouldParseCoalesceRowsDelaySeconds() { + Map map = new HashMap<>(); + map.put("coalesceRows.delaySeconds", "123"); + JobOptions options = OptionsParser.parse(map, JobOptions.class); + assertEquals(options.getCoalesceRowsDelaySeconds(), 123L); + } + + @Test + public void test_shouldParseCoalesceRowsEnabled() { + Map map = new HashMap<>(); + map.put("coalesceRows.enabled", "true"); + JobOptions options = OptionsParser.parse(map, JobOptions.class); + assertTrue(options.isCoalesceRowsEnabled()); + + } + + + @Test + public void test_shouldParseCoalesceRowsTimeoutSeconds() { + Map map = new HashMap<>(); + map.put("coalesceRows.timeoutSeconds", "1800"); + JobOptions options = OptionsParser.parse(map, JobOptions.class); + assertEquals(options.getCoalesceRowsTimeoutSeconds(), 1800L); + } +} \ No newline at end of file diff --git a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java index bc0b91cefcb..33e80be63b2 100644 --- a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java +++ b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java @@ -150,7 +150,7 @@ public void testStream_shouldMergeFeatures() { .advanceWatermarkToInfinity(); PCollection input = pipeline.apply(testStream); - PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + PCollection output = input.apply(new CoalesceFeatureRows(delay, Duration.ZERO)); PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); PAssert.that(output).containsInAnyOrder( @@ -183,7 +183,7 @@ public void testStream_shouldMergeFeatures_emittingPanes() { PCollection input = pipeline.apply(testStream); PCollection output = input - .apply(new CoalesceFeatureRows().withDelay(delay)); + .apply(new CoalesceFeatureRows(delay, Duration.ZERO)); PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( @@ -224,7 +224,7 @@ public void testStream_shouldIncludeRowOnTimerEdge() { PCollection input = pipeline.apply(testStream); PCollection output = input - .apply(new CoalesceFeatureRows().withDelay(delay)); + .apply(new CoalesceFeatureRows(delay, Duration.ZERO)); PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( @@ -264,7 +264,7 @@ public void testStream_shouldMergeFeatures_emittingPanes_overlappingTimers() { .advanceWatermarkToInfinity(); PCollection input = pipeline.apply(testStream); - PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + PCollection output = input.apply(new CoalesceFeatureRows(delay, Duration.ZERO)); PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( @@ -304,7 +304,7 @@ public void testStream_shouldNotSetTimerWhilePending() { .advanceWatermarkToInfinity(); PCollection input = pipeline.apply(testStream); - PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + PCollection output = input.apply(new CoalesceFeatureRows(delay, Duration.ZERO)); PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( @@ -351,7 +351,28 @@ public void test_shouldPickLatestFeatures() { } @Test - public void testStream_withNoDelay_shouldNotTriggerBufferTimer() { + public void testStream_withNoInput() { + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows()); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(0L)); + pipeline.run(); + } + + @Test + public void testBatch_withNoInput() { + PCollection input = pipeline.apply(Create.empty(ProtoCoder.of(FeatureRow.class))); + PCollection output = input.apply(new CoalesceFeatureRows()); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(0L)); + pipeline.run(); + } + + @Test + public void testStream_withTimeout_shouldRemoveState() { List rows = Lists.newArrayList( FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) @@ -361,21 +382,26 @@ public void testStream_withNoDelay_shouldNotTriggerBufferTimer() { .build() ); - Duration delay = Duration.standardSeconds(0); + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + Duration timeout = Duration.standardMinutes(30); TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) .addElements(rows.get(0)) - // Even though we have no delay, we should still get a single output row as processing time has not advanced + .advanceWatermarkTo(start.plus(timeout)) + // first element should get fired, as the delay water mark is reached before the timeout + // watermark, then state should be cleared when it reaches the timeout watermark. .addElements(rows.get(1)) - .advanceProcessingTime(delay.plus(1)) .advanceWatermarkToInfinity(); PCollection input = pipeline.apply(testStream); - PCollection output = input.apply(new CoalesceFeatureRows().withDelay(delay)); + PCollection output = input.apply(new CoalesceFeatureRows(delay, timeout)); - PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); + PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) .build() ); @@ -383,23 +409,37 @@ public void testStream_withNoDelay_shouldNotTriggerBufferTimer() { } @Test - public void testStream_withNoInput() { + public void testStream_withDelayAfterTimeout_shouldProcessBagBeforeClear() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .build(), + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); + + Instant start = new Instant(); + Duration delay = Duration.standardMinutes(40); + Duration timeout = Duration.standardMinutes(30); TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .addElements(rows.get(0)) + .addElements(rows.get(1)) + // first element should get fired, as the delay water mark is reached before the timeout + // watermark, then state should be cleared when it reaches the timeout watermark. + // If it didn't process the bag before clearing it, we'd get no output events at all. .advanceWatermarkToInfinity(); PCollection input = pipeline.apply(testStream); - PCollection output = input.apply(new CoalesceFeatureRows()); + PCollection output = input.apply(new CoalesceFeatureRows(delay, timeout)); - PAssert.that(output).satisfies(FeastMatchers.hasCount(0L)); - pipeline.run(); - } - - @Test - public void testBatch_withNoInput() { - PCollection input = pipeline.apply(Create.empty(ProtoCoder.of(FeatureRow.class))); - PCollection output = input.apply(new CoalesceFeatureRows()); - - PAssert.that(output).satisfies(FeastMatchers.hasCount(0L)); + PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); + PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( + FeatureRow.newBuilder().setEntityKey("1") + .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build() + ); pipeline.run(); } } \ No newline at end of file diff --git a/ingestion/src/test/java/feast/ingestion/transform/ErrorsStoreTransformTest.java b/ingestion/src/test/java/feast/ingestion/transform/ErrorsStoreTransformTest.java index 7255c42f225..83c0ef3e025 100644 --- a/ingestion/src/test/java/feast/ingestion/transform/ErrorsStoreTransformTest.java +++ b/ingestion/src/test/java/feast/ingestion/transform/ErrorsStoreTransformTest.java @@ -21,7 +21,7 @@ import static feast.storage.MockErrorsStore.MOCK_ERRORS_STORE_TYPE; import feast.ingestion.model.Specs; -import feast.ingestion.options.ImportJobOptions; +import feast.ingestion.options.ImportJobPipelineOptions; import feast.storage.MockErrorsStore; import feast.storage.service.ErrorsStoreService; import feast.storage.stderr.StderrErrorsStore; @@ -49,14 +49,14 @@ public class ErrorsStoreTransformTest { @Rule public TestPipeline pipeline = TestPipeline.create(); - private ImportJobOptions options; + private ImportJobPipelineOptions options; private Specs specs; private PCollection inputs; private List errors; @Before public void setUp() { - options = PipelineOptionsFactory.create().as(ImportJobOptions.class); + options = PipelineOptionsFactory.create().as(ImportJobPipelineOptions.class); options.setJobName("test"); specs = Specs.builder().jobName("test").build(); diff --git a/protos/feast/specs/ImportSpec.proto b/protos/feast/specs/ImportSpec.proto index 53838df3338..1afc6c26d1c 100644 --- a/protos/feast/specs/ImportSpec.proto +++ b/protos/feast/specs/ImportSpec.proto @@ -25,23 +25,26 @@ option go_package = "github.com/gojek/feast/protos/generated/go/feast/specs"; import "google/protobuf/timestamp.proto"; message ImportSpec { - // type should define what sort of store it is - // pubsub: topic, subscription (already a featureRow) - // kafka: ? (already a featureRow) - // file: csv, parquet, json - // bigquery: table, query string type = 1; map options = 2; + map jobOptions = 5; + repeated string entities = 3; Schema schema = 4; } +message ImportJobOptions { + bool coalesceRows = 1; + int64 coalesceRowsDelaySeconds = 2; +} + message Schema { repeated Field fields = 1; - oneof timestamp { // the event timestamp to set per row. + // the event timestamp to set per row. + oneof timestamp { string timestampColumn = 5; google.protobuf.Timestamp timestampValue = 6; } From d34e2985d47ab2d3194036366cc5c856324435b5 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Thu, 24 Jan 2019 18:41:18 +0700 Subject: [PATCH 03/15] coalesce for serving on entity name and key only, preparing for removing history from serving stores --- .../main/java/feast/ingestion/ImportJob.java | 41 +-- .../feast/ingestion/metrics/FeastMetrics.java | 76 ++--- .../metrics/FeastMetricsPubSubSink.java | 110 ------- .../transform/CoalesceFeatureRows.java | 145 ++++++---- .../bigquery/BigQueryFeatureSource.java | 2 +- .../source/csv/CsvFileFeatureSource.java | 5 +- .../source/json/JsonFileFeatureSource.java | 2 +- .../source/kafka/KafkaFeatureSource.java | 2 +- .../source/pubsub/PubSubFeatureSource.java | 2 +- .../feast_ingestion/types/CoalesceAccum.proto | 37 +++ .../feast_ingestion/types/CoalesceKey.proto | 25 ++ .../proto/feast_ingestion/types/README.md | 1 + .../test/java/feast/NormalizeFeatureRows.java | 15 +- .../feast/ingestion/ImportJobCSVTest.java | 110 ++++++- .../config/ImportSpecSupplierTest.java | 16 +- .../transform/CoalesceFeatureRowsTest.java | 271 +++++++++++++----- protos/feast/specs/ImportSpec.proto | 2 +- 17 files changed, 501 insertions(+), 361 deletions(-) delete mode 100644 ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java create mode 100644 ingestion/src/main/proto/feast_ingestion/types/CoalesceAccum.proto create mode 100644 ingestion/src/main/proto/feast_ingestion/types/CoalesceKey.proto create mode 100644 ingestion/src/main/proto/feast_ingestion/types/README.md diff --git a/ingestion/src/main/java/feast/ingestion/ImportJob.java b/ingestion/src/main/java/feast/ingestion/ImportJob.java index f757a6f4471..802192c0fd2 100644 --- a/ingestion/src/main/java/feast/ingestion/ImportJob.java +++ b/ingestion/src/main/java/feast/ingestion/ImportJob.java @@ -18,7 +18,6 @@ package feast.ingestion; import com.google.api.services.bigquery.model.TableRow; -import com.google.common.collect.Lists; import com.google.inject.Guice; import com.google.inject.Inject; import com.google.inject.Injector; @@ -46,7 +45,6 @@ import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; import java.util.Arrays; -import java.util.List; import java.util.Random; import lombok.extern.slf4j.Slf4j; import org.apache.beam.runners.dataflow.DataflowPipelineJob; @@ -58,7 +56,6 @@ import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sample; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; @@ -66,7 +63,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.commons.codec.digest.DigestUtils; import org.joda.time.DateTime; @@ -152,7 +148,6 @@ public void expand() { } catch (InvalidProtocolBufferException e) { // pass } - specs.validate(); PCollection features = pipeline.apply("Read", readFeaturesTransform); @@ -175,28 +170,23 @@ public void expand() { ParDo.of(new RoundEventTimestampsDoFn())), pFeatureRows.getErrors()); + log.info( + "A sample of size 1 of incoming rows from MAIN and ERRORS will logged every 30 seconds for visibility"); + logNRows(pFeatureRows, "Output sample", 1, Duration.standardSeconds(30)); + + PFeatureRows servingRows = pFeatureRows; + PFeatureRows warehouseRows = pFeatureRows; + if (jobOptions.isCoalesceRowsEnabled()) { - pFeatureRows = pFeatureRows.apply("foo", new CoalescePFeatureRows( + // Should we merge and dedupe rows before writing to the serving store? + servingRows = servingRows.apply("Coalesce Rows", new CoalescePFeatureRows( jobOptions.getCoalesceRowsDelaySeconds(), jobOptions.getCoalesceRowsTimeoutSeconds())); } - if (!dryRun) { - List> errors = Lists.newArrayList(); - pFeatureRows = pFeatureRows.apply("Write to Serving Stores", servingStoreTransform); - errors.add(pFeatureRows.getErrors()); - pFeatureRows = PFeatureRows.of(pFeatureRows.getMain()); - - log.info( - "A sample of any 2 rows from each of MAIN, RETRIES and ERRORS will logged for convenience"); - logNRows(pFeatureRows, "Output sample", 2); - - PFeatureRows.of(pFeatureRows.getMain()) - .apply("Write to Warehouse Stores", warehouseStoreTransform); - errors.add(pFeatureRows.getErrors()); - - PCollectionList.of(errors).apply("flatten errors", Flatten.pCollections()) - .apply("Write serving errors", errorsStoreTransform); + servingRows.apply("Write to Serving Stores", servingStoreTransform); + warehouseRows.apply("Write to Warehouse Stores", warehouseStoreTransform); + pFeatureRows.getErrors().apply("Write errors", errorsStoreTransform); } } @@ -206,17 +196,16 @@ public PipelineResult run() { return result; } - public void logNRows(PFeatureRows pFeatureRows, String name, int limit) { + public void logNRows(PFeatureRows pFeatureRows, String name, long limit, Duration period) { PCollection main = pFeatureRows.getMain(); PCollection errors = pFeatureRows.getErrors(); if (main.isBounded().equals(IsBounded.UNBOUNDED)) { Window minuteWindow = - Window.into(FixedWindows.of(Duration.standardMinutes(1L))) + Window.into(FixedWindows.of(period)) .triggering(AfterWatermark.pastEndOfWindow()) .discardingFiredPanes() - .withAllowedLateness(Duration.standardMinutes(1)); - + .withAllowedLateness(Duration.ZERO); main = main.apply(minuteWindow); errors = errors.apply(minuteWindow); } diff --git a/ingestion/src/main/java/feast/ingestion/metrics/FeastMetrics.java b/ingestion/src/main/java/feast/ingestion/metrics/FeastMetrics.java index d6463e7e5c7..c357454dbdf 100644 --- a/ingestion/src/main/java/feast/ingestion/metrics/FeastMetrics.java +++ b/ingestion/src/main/java/feast/ingestion/metrics/FeastMetrics.java @@ -17,34 +17,22 @@ package feast.ingestion.metrics; -import feast.types.FeatureRowProto; -import lombok.AllArgsConstructor; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.transforms.DoFn; -import org.joda.time.DateTime; -import feast.ingestion.model.Features; -import feast.ingestion.model.Values; import feast.ingestion.util.DateUtil; import feast.types.FeatureProto.Feature; import feast.types.FeatureRowExtendedProto.FeatureRowExtended; +import feast.types.FeatureRowProto; import feast.types.FeatureRowProto.FeatureRow; -import feast.types.GranularityProto.Granularity; -import feast.types.GranularityProto.Granularity.Enum; - import java.time.Instant; -import java.util.concurrent.TimeUnit; +import lombok.AllArgsConstructor; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; public class FeastMetrics { - public static final String METRICS_ENTITY_NAME = "metrics"; - public static final String METRICS_FEATURE_JOB_ID = "metrics.hour.job_id"; - public static final String METRICS_FEATURE_NAMESPACE = "metrics.hour.namespace"; - public static final String METRICS_FEATURE_STEP = "metrics.hour.step"; - public static final String METRICS_FEATURE_NAME = "metrics.hour.name"; - public static final String METRICS_FEATURE_ATTEMPTED = "metrics.hour.attempted"; + public static final String FEAST_NAMESPACE = "feast"; - private FeastMetrics() {} + private FeastMetrics() { + } private static void inc(String name) { Metrics.counter(FeastMetrics.FEAST_NAMESPACE, name).inc(); @@ -70,41 +58,9 @@ public static CalculateLagMetricFunc lagUpdateDoFn() { return new CalculateLagMetricFunc(); } - /** - * Create a feature row from Metrics. - * The granularity is unrelated to the metrics themselves, and simply indicates the - * granularity at which to store and overwrite the metrics downstream. - * - * @param counter - * @param jobName - * @param granularity - * @return - */ - public static FeatureRow makeFeatureRow( - MetricResult counter, String jobName, Granularity.Enum granularity) { - String jobId = jobName; - String namespace = counter.getName().getNamespace(); - String step = counter.getStep(); - String name = counter.getName().getName(); - Long attempted = counter.getAttempted(); - - String entityId = String.join(":", new String[] {jobId, namespace, step, name}); - - return FeatureRow.newBuilder() - .setEntityName(METRICS_ENTITY_NAME) - .setEntityKey(entityId) - .setGranularity(granularity) - .setEventTimestamp(DateUtil.toTimestamp(DateTime.now())) - .addFeatures(Features.of(METRICS_FEATURE_JOB_ID, Values.ofString(jobId))) - .addFeatures(Features.of(METRICS_FEATURE_NAMESPACE, Values.ofString(namespace))) - .addFeatures(Features.of(METRICS_FEATURE_STEP, Values.ofString(step))) - .addFeatures(Features.of(METRICS_FEATURE_NAME, Values.ofString(name))) - .addFeatures(Features.of(METRICS_FEATURE_ATTEMPTED, Values.ofInt64(attempted))) - .build(); - } - @AllArgsConstructor public static class IncrRowExtendedFunc extends DoFn { + private String suffix; @ProcessElement @@ -117,18 +73,20 @@ public void processElement( @AllArgsConstructor public static class CalculateLagMetricFunc extends DoFn { + @ProcessElement - public void processElement(@Element FeatureRowExtended element, OutputReceiver out) { + public void processElement(@Element FeatureRowExtended element, + OutputReceiver out) { FeatureRowProto.FeatureRow row = element.getRow(); com.google.protobuf.Timestamp eventTimestamp = row.getEventTimestamp(); Instant now = Instant.now(); com.google.protobuf.Timestamp roundedCurrentTimestamp = - DateUtil.roundToGranularity( - com.google.protobuf.Timestamp.newBuilder() - .setSeconds(now.getEpochSecond()) - .setNanos(now.getNano()) - .build(), - row.getGranularity()); + DateUtil.roundToGranularity( + com.google.protobuf.Timestamp.newBuilder() + .setSeconds(now.getEpochSecond()) + .setNanos(now.getNano()) + .build(), + row.getGranularity()); long lagSeconds = roundedCurrentTimestamp.getSeconds() - eventTimestamp.getSeconds(); FeastMetrics.update("row:lag", lagSeconds); out.output(element); diff --git a/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java b/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java deleted file mode 100644 index 5bf79e20df0..00000000000 --- a/ingestion/src/main/java/feast/ingestion/metrics/FeastMetricsPubSubSink.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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.metrics; - -import com.google.auto.service.AutoService; -import com.google.common.base.Preconditions; -import com.google.protobuf.util.JsonFormat; -import java.io.IOException; -import java.util.Collections; -import java.util.Optional; -import lombok.extern.slf4j.Slf4j; -import org.apache.beam.sdk.io.gcp.pubsub.FeastPubsubHelper; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath; -import org.apache.beam.sdk.metrics.MetricQueryResults; -import org.apache.beam.sdk.metrics.MetricResult; -import org.apache.beam.sdk.metrics.MetricsSink; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import feast.ingestion.options.ImportJobPipelineOptions; -import feast.types.FeatureRowProto.FeatureRow; -import feast.types.GranularityProto.Granularity; - -/** - * MetricsSink class for dogfooding metrics back into feast as FeatureRows with Google Cloud PubSub. - */ -@Slf4j -public class FeastMetricsPubSubSink implements MetricsSink { - - private FeastPubsubHelper pubsubHelper; - private String jobName; - private String topicUrl; - private Granularity.Enum granularity; - - public FeastMetricsPubSubSink(PipelineOptions pipelineOptions) { - this.pubsubHelper = new FeastPubsubHelper(pipelineOptions); - this.jobName = pipelineOptions.getJobName(); - FeastMetricsPubSubSinkOptions sinkOptions = - pipelineOptions.as(FeastMetricsPubSubSinkOptions.class); - this.topicUrl = sinkOptions.getMetricsPubSubSinkTopicUrl(); - - String granularityString = - Optional.ofNullable(sinkOptions.getMetricsPubSubSinkGranularity()).orElse("HOUR"); - this.granularity = Granularity.Enum.valueOf(granularityString.toUpperCase()); - - Preconditions.checkNotNull(topicUrl, "FeastMetricsPubSubSink requires pubsub topic url"); - } - - /** - * Publish metrics as FeatureRows to PubSub - * - * @param metricQueryResults - * @throws Exception - */ - @Override - public void writeMetrics(MetricQueryResults metricQueryResults) throws Exception { - try { - for (MetricResult counter : metricQueryResults.getCounters()) { - FeatureRow row = FeastMetrics.makeFeatureRow(counter, jobName, granularity); - log.info(JsonFormat.printer().print(row)); - publish(topicUrl, row); - } - } catch (Throwable e) { - log.error(e.getMessage(), e); - } - } - - void publish(String pubSubTopicUrl, FeatureRow row) throws IOException { - TopicPath topicPath = PubsubClient.topicPathFromPath(pubSubTopicUrl); - pubsubHelper.publish(topicPath, row); - } - - public interface FeastMetricsPubSubSinkOptions extends PipelineOptions { - @Description("PubSub topic to write stats to") - String getMetricsPubSubSinkTopicUrl(); - - void setMetricsPubSubSinkTopicUrl(String value); - - @Description( - "Granularity to be set on published metrics FeatureRows, this effects storage" - + " downstream,it does not indicate any aggregation as all metrics are totals.") - String getMetricsPubSubSinkGranularity(); - - void setMetricsPubSubSinkGranularity(String value); - } - - @AutoService(PipelineOptionsRegistrar.class) - public static class FeastMetricsPubSubSinkOptionsRegistrar implements PipelineOptionsRegistrar { - @Override - public Iterable> getPipelineOptions() { - return Collections.singleton(ImportJobPipelineOptions.class); - } - } -} diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java index 4bc5f172553..00cc2534d7c 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java @@ -18,15 +18,16 @@ package feast.ingestion.transform; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; import com.google.protobuf.Timestamp; import feast.types.FeatureProto.Feature; import feast.types.FeatureRowProto.FeatureRow; -import feast.types.FeatureRowProto.FeatureRowKey; -import java.util.Collections; +import feast_ingestion.types.CoalesceAccumProto.CoalesceAccum; +import feast_ingestion.types.CoalesceKeyProto.CoalesceKey; import java.util.Comparator; import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import lombok.AllArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.beam.sdk.coders.KvCoder; @@ -68,12 +69,10 @@ public class CoalesceFeatureRows extends private static final Comparator TIMESTAMP_COMPARATOR = Comparator .comparing(Timestamp::getSeconds) .thenComparing(Timestamp::getNanos); - private static final SerializableFunction KEY_FUNCTION = (row) -> - FeatureRowKey.newBuilder() + private static final SerializableFunction KEY_FUNCTION = (row) -> + CoalesceKey.newBuilder() .setEntityName(row.getEntityName()) - .setEntityKey(row.getEntityKey()) - .setGranularity(row.getGranularity()) - .setEventTimestamp(row.getEventTimestamp()).build(); + .setEntityKey(row.getEntityKey()).build(); private static final Duration DEFAULT_DELAY = Duration.standardSeconds(10); private static final Duration DEFAULT_TIMEOUT = Duration.ZERO; @@ -95,54 +94,86 @@ public CoalesceFeatureRows(Duration delay, Duration timeout) { this.timeout = (timeout.isEqual(Duration.ZERO)) ? DEFAULT_TIMEOUT : timeout; } + /** + * Return a FeatureRow of the new features accumulated since the given timestamp + */ + public static FeatureRow toFeatureRow(CoalesceAccum accum, long counter) { + Preconditions.checkArgument(counter <= + accum.getCounter(), "Accumulator has no features at or newer than the provided counter"); + FeatureRow.Builder builder = FeatureRow.newBuilder() + .setEntityName(accum.getEntityName()) + .setGranularity(accum.getGranularity()) + .setEntityKey(accum.getEntityKey()) + // This will be the latest timestamp + .setEventTimestamp(accum.getEventTimestamp()); + + Map features = accum.getFeaturesMap(); + if (counter <= 0) { + builder.addAllFeatures(features.values()); + } else { + List featureList = accum.getFeatureMarksMap().entrySet().stream() + .filter((e) -> e.getValue() > counter) + .map((e) -> features.get(e.getKey())) + .collect(Collectors.toList()); + builder.addAllFeatures(featureList); + + } + return builder.build(); + } public static FeatureRow combineFeatureRows(Iterable rows) { - FeatureRow latestRow = null; + return toFeatureRow(combineFeatureRows(CoalesceAccum.getDefaultInstance(), rows), 0); + } + + public static CoalesceAccum combineFeatureRows(CoalesceAccum seed, Iterable rows) { + CoalesceAccum.Builder accum = seed.toBuilder(); Map features = new HashMap<>(); - int rowCount = 0; + Map featureMarks = new HashMap<>(); + long rowCount = seed.getCounter(); for (FeatureRow row : rows) { rowCount += 1; - if (latestRow == null) { - latestRow = row; + if (TIMESTAMP_COMPARATOR.compare(accum.getEventTimestamp(), row.getEventTimestamp()) + <= 0) { + // row has later timestamp than accum. + for (Feature feature : row.getFeaturesList()) { + features.put(feature.getId(), feature); + // These marks are used to determine which features are new when we convert an accum + // back into a FeatureRow. + featureMarks.put(feature.getId(), rowCount); + } + accum.setEntityName(row.getEntityName()); + accum.setEntityKey(row.getEntityKey()); + accum.setGranularity(row.getGranularity()); + accum.setEventTimestamp(row.getEventTimestamp()); } else { - if (TIMESTAMP_COMPARATOR.compare(latestRow.getEventTimestamp(), row.getEventTimestamp()) - < 0) { - // row has later timestamp than agg. - for (Feature feature : row.getFeaturesList()) { - features.put(feature.getId(), feature); - } - latestRow = row; - } else { - for (Feature feature : row.getFeaturesList()) { - String featureId = feature.getId(); - // only insert an older feature if there was no newer one. - if (!features.containsKey(featureId)) { - features.put(featureId, feature); - } + for (Feature feature : row.getFeaturesList()) { + String featureId = feature.getId(); + // only insert an older feature if there was no newer one. + if (!features.containsKey(featureId)) { + features.put(featureId, feature); } } } } - - Preconditions.checkNotNull(latestRow); - if (rowCount == 1) { - return latestRow; + if (rowCount == seed.getCounter()) { + return seed; } else { - for (Feature feature : latestRow.getFeaturesList()) { - features.put(feature.getId(), feature); - } - return latestRow.toBuilder().clearFeatures().addAllFeatures(features.values()).build(); + return accum + .setCounter(rowCount) + .putAllFeatures(features) + .putAllFeatureMarks(featureMarks) + .build(); } } @Override public PCollection expand(PCollection input) { - PCollection> kvs = input - .apply(WithKeys.of(KEY_FUNCTION).withKeyType(TypeDescriptor.of(FeatureRowKey.class))) - .setCoder(KvCoder.of(ProtoCoder.of(FeatureRowKey.class), ProtoCoder.of(FeatureRow.class))); + PCollection> kvs = input + .apply(WithKeys.of(KEY_FUNCTION).withKeyType(TypeDescriptor.of(CoalesceKey.class))) + .setCoder(KvCoder.of(ProtoCoder.of(CoalesceKey.class), ProtoCoder.of(FeatureRow.class))); if (kvs.isBounded().equals(IsBounded.UNBOUNDED)) { - return kvs.apply("Configure window", Window.>configure() + return kvs.apply("Configure window", Window.>configure() .withAllowedLateness(Duration.ZERO) .discardingFiredPanes() .triggering(AfterProcessingTime.pastFirstElementInPane())) @@ -154,14 +185,15 @@ public PCollection expand(PCollection input) { } } + @Slf4j @AllArgsConstructor public static class CombineStateDoFn extends - DoFn, KV> { + DoFn, KV> { - @StateId("lastKnownValue") - private final StateSpec> lastKnownValue = - StateSpecs.value(ProtoCoder.of(FeatureRow.class)); + @StateId("lastKnownAccumValue") + private final StateSpec> lastKnownAccumValueSpecs = + StateSpecs.value(ProtoCoder.of(CoalesceAccum.class)); @StateId("newElementsBag") private final StateSpec> newElementsBag = StateSpecs.bag(ProtoCoder.of(FeatureRow.class)); @@ -201,44 +233,45 @@ public void processElement(ProcessContext context, @OnTimer("bufferTimer") public void bufferOnTimer( - OnTimerContext context, OutputReceiver> out, + OnTimerContext context, OutputReceiver> out, @StateId("newElementsBag") BagState newElementsBag, - @StateId("lastKnownValue") ValueState lastKnownValue) { + @StateId("lastKnownAccumValue") ValueState lastKnownAccumValue) { log.debug("bufferOnTimer triggered {}", context.timestamp()); - flush(out, newElementsBag, lastKnownValue); + flush(out, newElementsBag, lastKnownAccumValue); } @OnTimer("timeoutTimer") public void timeoutOnTimer( - OnTimerContext context, OutputReceiver> out, + OnTimerContext context, OutputReceiver> out, @StateId("newElementsBag") BagState newElementsBag, - @StateId("lastKnownValue") ValueState lastKnownValue) { + @StateId("lastKnownAccumValue") ValueState lastKnownAccumValue) { log.debug("timeoutOnTimer triggered {}", context.timestamp()); - flush(out, newElementsBag, lastKnownValue); + flush(out, newElementsBag, lastKnownAccumValue); newElementsBag.clear(); - lastKnownValue.clear(); + lastKnownAccumValue.clear(); } public void flush( - OutputReceiver> out, + OutputReceiver> out, @StateId("newElementsBag") BagState newElementsBag, - @StateId("lastKnownValue") ValueState lastKnownValue) { + @StateId("lastKnownAccumValue") ValueState lastKnownAccumValue) { log.debug("Flush triggered"); Iterable rows = newElementsBag.read(); if (!rows.iterator().hasNext()) { log.debug("Flush with no new elements"); return; } - FeatureRow lastKnown = lastKnownValue.read(); - if (lastKnown != null) { - rows = Iterables.concat(Collections.singleton(lastKnown), newElementsBag.read()); + CoalesceAccum lastKnownAccum = lastKnownAccumValue.read(); + if (lastKnownAccum == null) { + lastKnownAccum = CoalesceAccum.getDefaultInstance(); } // Check if we have more than one value in our list. - FeatureRow row = combineFeatureRows(rows); + CoalesceAccum accum = combineFeatureRows(lastKnownAccum, rows); + FeatureRow row = toFeatureRow(accum, lastKnownAccum.getCounter()); log.debug("Timer fired and added FeatureRow to output {}", row); // Clear the elements now that they have been processed newElementsBag.clear(); - lastKnownValue.write(row); + lastKnownAccumValue.write(accum); // Output the value stored in the the processed que which matches this timers time out.output(KV.of(KEY_FUNCTION.apply(row), row)); diff --git a/ingestion/src/main/java/feast/source/bigquery/BigQueryFeatureSource.java b/ingestion/src/main/java/feast/source/bigquery/BigQueryFeatureSource.java index 9ce7f965f2c..b183ffbddc0 100644 --- a/ingestion/src/main/java/feast/source/bigquery/BigQueryFeatureSource.java +++ b/ingestion/src/main/java/feast/source/bigquery/BigQueryFeatureSource.java @@ -65,7 +65,7 @@ public class BigQueryFeatureSource extends FeatureSource { @Override public PCollection expand(PInput input) { BigQuerySourceOptions options = OptionsParser - .parse(importSpec.getOptionsMap(), BigQuerySourceOptions.class); + .parse(importSpec.getSourceOptionsMap(), BigQuerySourceOptions.class); List entities = importSpec.getEntitiesList(); Preconditions.checkArgument( diff --git a/ingestion/src/main/java/feast/source/csv/CsvFileFeatureSource.java b/ingestion/src/main/java/feast/source/csv/CsvFileFeatureSource.java index 837050fc680..e027a25d1b1 100644 --- a/ingestion/src/main/java/feast/source/csv/CsvFileFeatureSource.java +++ b/ingestion/src/main/java/feast/source/csv/CsvFileFeatureSource.java @@ -70,7 +70,7 @@ public class CsvFileFeatureSource extends FeatureSource { @Override public PCollection expand(PInput input) { CsvFileFeatureSourceOptions options = OptionsParser - .parse(importSpec.getOptionsMap(), CsvFileFeatureSourceOptions.class); + .parse(importSpec.getSourceOptionsMap(), CsvFileFeatureSourceOptions.class); List entities = importSpec.getEntitiesList(); Preconditions.checkArgument( entities.size() == 1, "exactly 1 entity must be set for CSV import"); @@ -109,6 +109,9 @@ public void processElement(ProcessContext context) { for (Entry entry : stringMap.entrySet()) { String name = entry.getKey(); String value = entry.getValue(); + if (value.isEmpty()) { + continue; + } Field field = fields.get(name); // A feature can only be one of these things diff --git a/ingestion/src/main/java/feast/source/json/JsonFileFeatureSource.java b/ingestion/src/main/java/feast/source/json/JsonFileFeatureSource.java index a38a2e0b8c1..08131c16f8b 100644 --- a/ingestion/src/main/java/feast/source/json/JsonFileFeatureSource.java +++ b/ingestion/src/main/java/feast/source/json/JsonFileFeatureSource.java @@ -56,7 +56,7 @@ public class JsonFileFeatureSource extends FeatureSource { @Override public PCollection expand(PInput input) { JsonFileFeatureSourceOptions options = OptionsParser - .parse(importSpec.getOptionsMap(), JsonFileFeatureSourceOptions.class); + .parse(importSpec.getSourceOptionsMap(), JsonFileFeatureSourceOptions.class); PCollection jsonLines = input.getPipeline().apply(TextIO.read().from(options.path)); return jsonLines.apply( ParDo.of( diff --git a/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java b/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java index d6d5e36ad01..9246c5a9fdc 100644 --- a/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java +++ b/ingestion/src/main/java/feast/source/kafka/KafkaFeatureSource.java @@ -59,7 +59,7 @@ public PCollection expand(PInput input) { checkArgument(importSpec.getType().equals(KAFKA_FEATURE_SOURCE_TYPE)); KafkaReadOptions options = - OptionsParser.parse(importSpec.getOptionsMap(), KafkaReadOptions.class); + OptionsParser.parse(importSpec.getSourceOptionsMap(), KafkaReadOptions.class); List topicsList = new ArrayList<>(Arrays.asList(options.topics.split(","))); diff --git a/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java b/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java index 925094828c6..c8bafbafc70 100644 --- a/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java +++ b/ingestion/src/main/java/feast/source/pubsub/PubSubFeatureSource.java @@ -60,7 +60,7 @@ public class PubSubFeatureSource extends FeatureSource { public PCollection expand(PInput input) { checkArgument(importSpec.getType().equals(PUBSUB_FEATURE_SOURCE_TYPE)); PubSubReadOptions options = - OptionsParser.parse(importSpec.getOptionsMap(), PubSubReadOptions.class); + OptionsParser.parse(importSpec.getSourceOptionsMap(), PubSubReadOptions.class); PubsubIO.Read read = readProtos(); diff --git a/ingestion/src/main/proto/feast_ingestion/types/CoalesceAccum.proto b/ingestion/src/main/proto/feast_ingestion/types/CoalesceAccum.proto new file mode 100644 index 00000000000..f8e82a59521 --- /dev/null +++ b/ingestion/src/main/proto/feast_ingestion/types/CoalesceAccum.proto @@ -0,0 +1,37 @@ +/* + * 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. + */ + +syntax = "proto3"; + +import "google/protobuf/timestamp.proto"; +import "feast/types/Feature.proto"; +import "feast/types/Granularity.proto"; + +option java_package = "feast_ingestion.types"; +option java_outer_classname = "CoalesceAccumProto"; + +// Accumlator for merging feature rows. +message CoalesceAccum { + string entityKey = 1; + google.protobuf.Timestamp eventTimestamp = 3; + string entityName = 4; + feast.types.Granularity.Enum granularity = 5; + + map features = 6; + // map of features to their counter values when they were last added to accumulator + map featureMarks = 7; + int64 counter = 8; +} \ No newline at end of file diff --git a/ingestion/src/main/proto/feast_ingestion/types/CoalesceKey.proto b/ingestion/src/main/proto/feast_ingestion/types/CoalesceKey.proto new file mode 100644 index 00000000000..9730b49ec3b --- /dev/null +++ b/ingestion/src/main/proto/feast_ingestion/types/CoalesceKey.proto @@ -0,0 +1,25 @@ +/* + * 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. + */ + +syntax = "proto3"; + +option java_package = "feast_ingestion.types"; +option java_outer_classname = "CoalesceKeyProto"; + +message CoalesceKey { + string entityName = 1; + string entityKey = 2; +} \ No newline at end of file diff --git a/ingestion/src/main/proto/feast_ingestion/types/README.md b/ingestion/src/main/proto/feast_ingestion/types/README.md new file mode 100644 index 00000000000..7736c97ba20 --- /dev/null +++ b/ingestion/src/main/proto/feast_ingestion/types/README.md @@ -0,0 +1 @@ +Internal protobuf messages used in ingestion \ No newline at end of file diff --git a/ingestion/src/test/java/feast/NormalizeFeatureRows.java b/ingestion/src/test/java/feast/NormalizeFeatureRows.java index 626f8800431..beb3b1b485b 100644 --- a/ingestion/src/test/java/feast/NormalizeFeatureRows.java +++ b/ingestion/src/test/java/feast/NormalizeFeatureRows.java @@ -20,7 +20,6 @@ import com.google.common.collect.Lists; import com.google.common.primitives.UnsignedBytes; import feast.types.FeatureProto.Feature; -import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; import java.util.List; import org.apache.beam.sdk.transforms.MapElements; @@ -31,12 +30,20 @@ public class NormalizeFeatureRows extends PTransform, PCollection> { - public static FeatureRow orderedFeatureRow(FeatureRow row) { + public static FeatureRow normalize(FeatureRow.Builder row) { + return normalize(row.build()); + } + + public static FeatureRow normalize(FeatureRow row) { List features = Lists.newArrayList(row.getFeaturesList()); features.sort( (f1, f2) -> UnsignedBytes.lexicographicalComparator().compare(f1.toByteArray(), f2.toByteArray())); - return row.toBuilder().clearFeatures().addAllFeatures(features).build(); + + return row.toBuilder() + .clearFeatures().addAllFeatures(features) + .setEventTimestamp(row.getEventTimestamp()) + .build(); } @Override @@ -45,6 +52,6 @@ public PCollection expand(PCollection input) { .apply( "normalize rows", MapElements.into(TypeDescriptor.of(FeatureRow.class)).via( - NormalizeFeatureRows::orderedFeatureRow)); + NormalizeFeatureRows::normalize)); } } diff --git a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java index e6741c648a9..38315591efc 100644 --- a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java +++ b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java @@ -18,7 +18,7 @@ package feast.ingestion; import static feast.FeastMatchers.hasCount; -import static feast.ToOrderedFeatureRows.orderedFeatureRow; +import static feast.NormalizeFeatureRows.normalize; import static feast.storage.MockErrorsStore.MOCK_ERRORS_STORE_TYPE; import static org.junit.Assert.assertEquals; @@ -73,12 +73,13 @@ public class ImportJobCSVTest { public TestPipeline testPipeline = TestPipeline.create(); public ImportSpec initImportSpec(ImportSpec importSpec, String dataFile) { - return importSpec.toBuilder().putOptions("path", dataFile).build(); + return importSpec.toBuilder().putSourceOptions("path", dataFile).build(); } public ImportJobPipelineOptions initOptions() { Path path = Paths.get(Resources.getResource("core_specs/").getPath()); - ImportJobPipelineOptions options = PipelineOptionsFactory.create().as(ImportJobPipelineOptions.class); + ImportJobPipelineOptions options = PipelineOptionsFactory.create() + .as(ImportJobPipelineOptions.class); options.setCoreApiSpecPath(path.toString()); options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); return options; @@ -90,7 +91,7 @@ public void testImportCSV() throws IOException { ProtoUtil.decodeProtoYaml( "---\n" + "type: file.csv\n" - + "options:\n" + + "sourceOptions:\n" + " # path: to be overwritten in tests\n" + "entities:\n" + " - testEntity\n" @@ -134,7 +135,7 @@ public void testImportCSV() throws IOException { List expectedRows = Lists.newArrayList( - orderedFeatureRow( + normalize( FeatureRow.newBuilder() .setGranularity(Granularity.Enum.NONE) .setEventTimestamp(Timestamp.getDefaultInstance()) @@ -143,7 +144,7 @@ public void testImportCSV() throws IOException { .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) .addFeatures(Features.of("testEntity.none.testString", Values.ofString("a"))) .build()), - orderedFeatureRow( + normalize( FeatureRow.newBuilder() .setGranularity(Granularity.Enum.NONE) .setEventTimestamp(Timestamp.getDefaultInstance()) @@ -152,7 +153,7 @@ public void testImportCSV() throws IOException { .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(202))) .addFeatures(Features.of("testEntity.none.testString", Values.ofString("b"))) .build()), - orderedFeatureRow( + normalize( FeatureRow.newBuilder() .setGranularity(Granularity.Enum.NONE) .setEventTimestamp(Timestamp.getDefaultInstance()) @@ -173,14 +174,99 @@ public void testImportCSV() throws IOException { testPipeline.run(); } + @Test + public void testImportCSV_withCoalesceRows() throws IOException { + ImportSpec importSpec = + ProtoUtil.decodeProtoYaml( + "---\n" + + "type: file.csv\n" + + "sourceOptions:\n" + + " # path: to be overwritten in tests\n" + + "jobOptions:\n" + + " coalesceRows.enabled: true\n" + + "entities:\n" + + " - testEntity\n" + + "schema:\n" + + " entityIdColumn: id\n" + + " timestampValue: 2018-09-25T00:00:00.000Z\n" + + " fields:\n" + + " - name: id\n" + + " - featureId: testEntity.none.testInt32\n" + + " - featureId: testEntity.none.testString\n" + + "\n", + ImportSpec.getDefaultInstance()); + + File csvFile = folder.newFile("data.csv"); + Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n1,,b\n"); + importSpec = initImportSpec(importSpec, csvFile.toString()); + + ImportJobPipelineOptions options = initOptions(); + options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); + + Injector injector = + Guice.createInjector( + new ImportJobModule(options, importSpec), new TestPipelineModule(testPipeline)); + + ImportJob job = injector.getInstance(ImportJob.class); + injector.getInstance(ImportJob.class); + job.expand(); + + PCollection writtenToServing = + PCollectionList.of(ServingStoreService.get(MockServingStore.class).getWrite().getInputs()) + .apply("flatten serving input", Flatten.pCollections()); + + PCollection writtenToWarehouse = + PCollectionList.of( + WarehouseStoreService.get(MockWarehouseStore.class).getWrite().getInputs()) + .apply("flatten warehouse input", Flatten.pCollections()); + + PCollection writtenToErrors = + PCollectionList.of(ErrorsStoreService.get(MockErrorsStore.class).getWrite().getInputs()) + .apply("flatten errors input", Flatten.pCollections()); + + PAssert.that(writtenToErrors).satisfies(hasCount(0)); + + PAssert.that(writtenToServing.apply("serving toFeatureRows", new ToOrderedFeatureRows())) + .containsInAnyOrder(normalize( + FeatureRow.newBuilder() + .setGranularity(Granularity.Enum.NONE) + .setEventTimestamp(Timestamp.getDefaultInstance()) + .setEntityKey("1") + .setEntityName("testEntity") + .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) + .addFeatures(Features.of("testEntity.none.testString", Values.ofString("b"))) + .build())); + + PAssert.that(writtenToWarehouse.apply("warehouse toFeatureRows", new ToOrderedFeatureRows())) + .containsInAnyOrder( + normalize( + FeatureRow.newBuilder() + .setGranularity(Granularity.Enum.NONE) + .setEventTimestamp(Timestamp.getDefaultInstance()) + .setEntityKey("1") + .setEntityName("testEntity") + .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) + .addFeatures(Features.of("testEntity.none.testString", Values.ofString("a"))) + .build()), + normalize( + FeatureRow.newBuilder() + .setGranularity(Granularity.Enum.NONE) + .setEventTimestamp(Timestamp.getDefaultInstance()) + .setEntityKey("1") + .setEntityName("testEntity") + .addFeatures(Features.of("testEntity.none.testString", Values.ofString("b"))) + .build())); + + testPipeline.run(); + } + @Test(expected = SpecRetrievalException.class) public void testImportCSVUnknownServingStoreError() throws IOException { ImportSpec importSpec = ProtoUtil.decodeProtoYaml( "---\n" + "type: file.csv\n" - + "options:\n" - + " format: csv\n" + + "sourceOptions:\n" + " # path: to be overwritten in tests\n" + "entities:\n" + " - testEntity\n" @@ -218,7 +304,7 @@ public void testImportWithErrors() throws IOException { ProtoUtil.decodeProtoYaml( "---\n" + "type: file.csv\n" - + "options:\n" + + "sourceOptions:\n" + " # path: to be overwritten in tests\n" + "entities:\n" + " - testEntity\n" @@ -283,7 +369,7 @@ public void testImportWithoutWarehouseStore() throws IOException { ProtoUtil.decodeProtoYaml( "---\n" + "type: file.csv\n" - + "options:\n" + + "sourceOptions:\n" + " # path: to be overwritten in tests\n" + "entities:\n" + " - testEntity\n" @@ -303,7 +389,7 @@ public void testImportWithoutWarehouseStore() throws IOException { Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n2,202,b\n3,303,c\n"); importSpec = initImportSpec(importSpec, csvFile.toString()); - ImportJobOptions options = initOptions(); + ImportJobPipelineOptions options = initOptions(); options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); Injector injector = diff --git a/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java b/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java index cf22ba523c5..00eacdcaeb4 100644 --- a/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java +++ b/ingestion/src/test/java/feast/ingestion/config/ImportSpecSupplierTest.java @@ -37,9 +37,8 @@ public class ImportSpecSupplierTest { String importSpecYaml = "---\n" - + "type: file\n" - + "options:\n" - + " format: csv\n" + + "type: file.csv\n" + + "sourceOptions:\n" + " path: data.csv\n" + "entities:\n" + " - driver\n" @@ -53,17 +52,10 @@ public class ImportSpecSupplierTest { + " featureId: driver.none.trips_completed\n" + "\n"; - String importSpecJson = - "{\"type\":\"file\",\"options\":{\"format\":\"csv\",\"path\":\"data.csv\"},\"entities\":[\"driver\"]," - + "\"schema\":" - + "{\"fields\":[{\"name\":\"timestamp\"}," - + "{\"name\":\"driver_id\"},{\"name\":\"trips_completed\",\"featureId\":\"driver.none.trips_completed\"}],\"timestampValue\":\"2018-09-25T00:00:00Z\",\"entityIdColumn\":\"driver_id\"}}\n"; - ImportSpec expectedImportSpec = ImportSpec.newBuilder() - .setType("file") - .putOptions("format", "csv") - .putOptions("path", "data.csv") + .setType("file.csv") + .putSourceOptions("path", "data.csv") .addEntities("driver") .setSchema( Schema.newBuilder() diff --git a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java index 33e80be63b2..11dba243c17 100644 --- a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java +++ b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java @@ -17,16 +17,21 @@ package feast.ingestion.transform; +import static feast.NormalizeFeatureRows.normalize; +import static feast.ingestion.transform.CoalesceFeatureRows.toFeatureRow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; import com.google.common.collect.Lists; import com.google.protobuf.Timestamp; import feast.FeastMatchers; import feast.NormalizeFeatureRows; +import feast.ingestion.model.Features; import feast.ingestion.model.Values; import feast.types.FeatureProto.Feature; import feast.types.FeatureRowProto.FeatureRow; +import feast_ingestion.types.CoalesceAccumProto.CoalesceAccum; import java.util.List; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.testing.PAssert; @@ -42,14 +47,20 @@ public class CoalesceFeatureRowsTest { + private static final Timestamp DEFAULT_TIMESTAMP = Timestamp.getDefaultInstance(); + private static final FeatureRow DEFAULT_FEATURE_ROW = FeatureRow.getDefaultInstance().toBuilder() + .setEventTimestamp(DEFAULT_TIMESTAMP).build(); + @Rule public TestPipeline pipeline = TestPipeline.create(); @Test public void testBatch_withDistictKeys_shouldPassThroughNonIntersectingKeys() { List rows = Lists.newArrayList( - FeatureRow.newBuilder().setEntityKey("1").build(), - FeatureRow.newBuilder().setEntityKey("2").build()); + FeatureRow.newBuilder().setEventTimestamp(Timestamp.getDefaultInstance()).setEntityKey("1") + .build(), + FeatureRow.newBuilder().setEventTimestamp(Timestamp.getDefaultInstance()).setEntityKey("2") + .build()); PCollection input = pipeline.apply(Create.of(rows)) .setCoder(ProtoCoder.of(FeatureRow.class)); @@ -65,8 +76,10 @@ public void testBatch_withDistictKeys_shouldPassThroughNonIntersectingKeys() { @Test public void test_withNoFeaturesSameTimestamp_shouldReturn1() { List rows = Lists.newArrayList( - FeatureRow.newBuilder().setEntityKey("1").build(), - FeatureRow.newBuilder().setEntityKey("1").build()); + FeatureRow.newBuilder().setEventTimestamp(Timestamp.getDefaultInstance()).setEntityKey("1") + .build(), + FeatureRow.newBuilder().setEventTimestamp(Timestamp.getDefaultInstance()).setEntityKey("1") + .build()); PCollection input = pipeline.apply(Create.of(rows)) .setCoder(ProtoCoder.of(FeatureRow.class)); @@ -123,6 +136,7 @@ public void testBatch_shouldMergeFeatures() { PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.getDefaultInstance()) .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) .build() @@ -155,6 +169,7 @@ public void testStream_shouldMergeFeatures() { PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); PAssert.that(output).containsInAnyOrder( FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.getDefaultInstance()) .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) .build() @@ -163,43 +178,7 @@ public void testStream_shouldMergeFeatures() { } @Test - public void testStream_shouldMergeFeatures_emittingPanes() { - List rows = Lists.newArrayList( - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .build(), - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) - .build()); - - Instant start = new Instant(); - Duration delay = Duration.standardSeconds(10); - TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) - .advanceWatermarkTo(start) - .addElements(rows.get(0)) - .advanceWatermarkTo(start.plus(delay).plus(delay.dividedBy(2))) - .addElements(rows.get(1)) - .advanceWatermarkToInfinity(); - - PCollection input = pipeline.apply(testStream); - PCollection output = input - .apply(new CoalesceFeatureRows(delay, Duration.ZERO)); - - PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); - PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) - .build(), - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .build() - ); - pipeline.run(); - } - - @Test - public void testStream_shouldIncludeRowOnTimerEdge() { + public void testStream_shouldIncludeRowAddedOnTimerEdge() { List rows = Lists.newArrayList( FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) @@ -216,7 +195,9 @@ public void testStream_shouldIncludeRowOnTimerEdge() { TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) .advanceWatermarkTo(start) .addElements(rows.get(0)) - .advanceWatermarkTo(start.plus(delay)) // Previous timer triggers + .advanceWatermarkTo(start.plus(delay)) + // This row will be included in the same pane because it's exactly + // on the same water mark as the onTimer event .addElements(rows.get(1)) .advanceWatermarkTo(start.plus(delay).plus(delay).plus(delay)) .addElements(rows.get(2)) @@ -228,15 +209,13 @@ public void testStream_shouldIncludeRowOnTimerEdge() { PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) - .build(), - FeatureRow.newBuilder().setEntityKey("1") + normalize(FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .build()), + normalize(FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) - .build() + .build()) ); pipeline.run(); } @@ -268,38 +247,27 @@ public void testStream_shouldMergeFeatures_emittingPanes_overlappingTimers() { PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( - FeatureRow.newBuilder().setEntityKey("1") + normalize(FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) - .build() + ) ); pipeline.run(); } @Test public void testStream_shouldNotSetTimerWhilePending() { - List rows = Lists.newArrayList( - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .build(), - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) - .build(), - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) - .build() - ); - Instant start = new Instant(); Duration delay = Duration.standardSeconds(10); TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) .advanceWatermarkTo(start) - .addElements(rows.get(0)) + .addElements(FeatureRow.getDefaultInstance()) + // this should not reset the timer as the first is still pending. .advanceWatermarkTo(start.plus(delay.dividedBy(2))) - .addElements(rows.get(1)) // this should not reset the timer as the first is still pending. + .addElements(FeatureRow.getDefaultInstance()) // timer should trigger causing the first output row .advanceWatermarkTo(start.plus(delay).plus(delay.dividedBy(2))) - .addElements(rows.get(2)) // this should cause a second output row. + .addElements(FeatureRow.getDefaultInstance()) // this should cause a second output row. .advanceWatermarkTo(start.plus(delay).plus(delay)) .advanceWatermarkToInfinity(); @@ -307,20 +275,104 @@ public void testStream_shouldNotSetTimerWhilePending() { PCollection output = input.apply(new CoalesceFeatureRows(delay, Duration.ZERO)); PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); - PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + PAssert.that(output).containsInAnyOrder(DEFAULT_FEATURE_ROW, DEFAULT_FEATURE_ROW); + pipeline.run(); + } + + @Test + public void testStream_shouldOnlyEmitNewFeaturesInSecondPane() { + Instant start = new Instant(); + Duration delay = Duration.standardSeconds(10); + TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) + .advanceWatermarkTo(start) + .addElements( + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("a"))) + .build()) + // this should should emit a row + .advanceWatermarkTo(start.plus(delay).plus(delay)) + .addElements( + FeatureRow.newBuilder() + .addFeatures(Features.of("f2", Values.ofString("b"))) + .build()) + // this should emit a row with f2 but without f1 because it hasn't had an update + .advanceWatermarkToInfinity(); + + PCollection input = pipeline.apply(testStream); + PCollection output = input.apply(new CoalesceFeatureRows(delay, Duration.ZERO)); + + PAssert.that(output).satisfies(FeastMatchers.hasCount(2L)); + PAssert.that(output).containsInAnyOrder( + FeatureRow.newBuilder() + .setEventTimestamp(DEFAULT_TIMESTAMP) + .addFeatures(Features.of("f1", Values.ofString("a"))) .build(), - FeatureRow.newBuilder().setEntityKey("1") - .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) - .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) - .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) - .build() - ); + FeatureRow.newBuilder() + .setEventTimestamp(DEFAULT_TIMESTAMP) + .addFeatures(Features.of("f2", Values.ofString("b"))) + .build()); pipeline.run(); } + @Test + public void test_combineFeatureRows_shouldCountRows() { + List rows = Lists.newArrayList( + FeatureRow.getDefaultInstance(), + FeatureRow.getDefaultInstance(), + FeatureRow.getDefaultInstance()); + CoalesceAccum accum = CoalesceFeatureRows + .combineFeatureRows(CoalesceAccum.getDefaultInstance(), rows); + assertEquals(3, accum.getCounter()); + } + + @Test + public void test_combineFeatureRows_shouldOverwriteWhenLaterEventTimestampProcessedSecond() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("a"))) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(1)) + .build(), + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("b"))) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) + .build()); + CoalesceAccum accum = CoalesceFeatureRows + .combineFeatureRows(CoalesceAccum.getDefaultInstance(), rows); + assertEquals(accum.getFeaturesMap().get("f1"), Features.of("f1", Values.ofString("b"))); + } + + @Test + public void test_combineFeatureRows_shouldNotOverwriteWhenEarlierEventTimestampProcessedSecond() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("b"))) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) + .build(), + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("a"))) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(1)) + .build()); + CoalesceAccum accum = CoalesceFeatureRows + .combineFeatureRows(CoalesceAccum.getDefaultInstance(), rows); + assertEquals(accum.getFeaturesMap().get("f1"), Features.of("f1", Values.ofString("b"))); + } + + @Test + public void test_combineFeatureRows_shouldOverwriteWhenSameEventTimestampProcessedSecond() { + List rows = Lists.newArrayList( + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("a"))) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) + .build(), + FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("b"))) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) + .build() + ); + CoalesceAccum accum = CoalesceFeatureRows + .combineFeatureRows(CoalesceAccum.getDefaultInstance(), rows); + assertEquals(accum.getFeaturesMap().get("f1"), Features.of("f1", Values.ofString("b"))); + } @Test public void test_shouldPickLatestFeatures() { @@ -340,7 +392,10 @@ public void test_shouldPickLatestFeatures() { .addFeatures(Feature.newBuilder().setId("f3").setValue(Values.ofInt32(3))) .build()); - assertThat(CoalesceFeatureRows.combineFeatureRows(rows), equalTo( + CoalesceAccum accum = CoalesceFeatureRows + .combineFeatureRows(CoalesceAccum.getDefaultInstance(), rows); + assertEquals(3, accum.getCounter()); + assertThat(toFeatureRow(accum, 0), equalTo( FeatureRow.newBuilder().setEntityKey("1") .setEventTimestamp(Timestamp.newBuilder().setSeconds(2)) .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(2))) @@ -400,9 +455,11 @@ public void testStream_withTimeout_shouldRemoveState() { PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) + .setEventTimestamp(Timestamp.getDefaultInstance()) .build(), FeatureRow.newBuilder().setEntityKey("1") .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) + .setEventTimestamp(Timestamp.getDefaultInstance()) .build() ); pipeline.run(); @@ -436,10 +493,72 @@ public void testStream_withDelayAfterTimeout_shouldProcessBagBeforeClear() { PAssert.that(output).satisfies(FeastMatchers.hasCount(1L)); PAssert.that(output.apply(new NormalizeFeatureRows())).containsInAnyOrder( FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(Timestamp.getDefaultInstance()) .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) .build() ); pipeline.run(); } + + @Test + public void test_toFeatureRow_shouldBeNewMarkedFeaturesOnly() { + CoalesceAccum accum = CoalesceAccum.newBuilder() + .putFeatures("f1", Features.of("f1", Values.ofString("a"))) + .putFeatures("f2", Features.of("f2", Values.ofString("b"))) + .putFeatures("f3", Features.of("f3", Values.ofString("c"))) + .putFeatures("f4", Features.of("f4", Values.ofString("d"))) + .putFeatureMarks("f1", 1) + .putFeatureMarks("f2", 1) + .putFeatureMarks("f3", 2) + .putFeatureMarks("f4", 3) + .setCounter(3) + .build(); + + FeatureRow output = normalize(toFeatureRow(accum, 0)); + assertThat(output, + equalTo(FeatureRow.newBuilder() + .addFeatures(Features.of("f1", Values.ofString("a"))) + .addFeatures(Features.of("f2", Values.ofString("b"))) + .addFeatures(Features.of("f3", Values.ofString("c"))) + .addFeatures(Features.of("f4", Values.ofString("d"))) + .setEventTimestamp(DEFAULT_TIMESTAMP) + .build())); + output = normalize(toFeatureRow(accum, 1)); + assertThat(output, + equalTo(FeatureRow.newBuilder() + .addFeatures(Features.of("f3", Values.ofString("c"))) + .addFeatures(Features.of("f4", Values.ofString("d"))) + .setEventTimestamp(DEFAULT_TIMESTAMP) + .build())); + output = normalize(toFeatureRow(accum, 2)); + assertThat(output, + equalTo(FeatureRow.newBuilder() + .addFeatures(Features.of("f4", Values.ofString("d"))) + .setEventTimestamp(DEFAULT_TIMESTAMP) + .build())); + output = normalize(toFeatureRow(accum, 3)); + assertThat(output, + equalTo(FeatureRow.newBuilder() + .setEventTimestamp(DEFAULT_TIMESTAMP) + .build())); + } + + @Test(expected = IllegalArgumentException.class) + public void test_toFeatureRow_markTooHigh_shouldThrow() { + CoalesceAccum accum = CoalesceAccum.newBuilder() + .putFeatures("f1", Features.of("f1", Values.ofString("a"))) + .putFeatures("f2", Features.of("f2", Values.ofString("b"))) + .putFeatures("f3", Features.of("f3", Values.ofString("c"))) + .putFeatures("f4", Features.of("f4", Values.ofString("d"))) + .putFeatureMarks("f1", 1) + .putFeatureMarks("f2", 1) + .putFeatureMarks("f3", 2) + .putFeatureMarks("f4", 3) + .setCounter(3) + .build(); + normalize(toFeatureRow(accum, 4)); + // we throw an exception because use case should check that we have new features before trying + // to emit them. + } } \ No newline at end of file diff --git a/protos/feast/specs/ImportSpec.proto b/protos/feast/specs/ImportSpec.proto index 1afc6c26d1c..a1b2d6273b8 100644 --- a/protos/feast/specs/ImportSpec.proto +++ b/protos/feast/specs/ImportSpec.proto @@ -27,7 +27,7 @@ import "google/protobuf/timestamp.proto"; message ImportSpec { string type = 1; - map options = 2; + map sourceOptions = 2; map jobOptions = 5; From 83d33a73afe2782c1eb5c5a55902468b4d7aee64 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Thu, 24 Jan 2019 20:08:11 +0700 Subject: [PATCH 04/15] change to rounding granularities after feature row coalesce --- .../main/java/feast/ingestion/ImportJob.java | 24 +++++++++------- .../transform/CoalesceFeatureRows.java | 13 ++++++--- .../transform/fn/ConvertTypesDoFn.java | 5 +++- .../source/csv/CsvFileFeatureSource.java | 6 +++- .../feast/ingestion/ImportJobCSVTest.java | 28 ++++++++++--------- .../transform/CoalesceFeatureRowsTest.java | 10 +++++-- 6 files changed, 54 insertions(+), 32 deletions(-) diff --git a/ingestion/src/main/java/feast/ingestion/ImportJob.java b/ingestion/src/main/java/feast/ingestion/ImportJob.java index 802192c0fd2..56977cf76e4 100644 --- a/ingestion/src/main/java/feast/ingestion/ImportJob.java +++ b/ingestion/src/main/java/feast/ingestion/ImportJob.java @@ -161,28 +161,22 @@ public void expand() { PFeatureRows pFeatureRows = PFeatureRows.of(featuresExtended); pFeatureRows = pFeatureRows.applyDoFn("Convert feature types", new ConvertTypesDoFn(specs)); pFeatureRows = pFeatureRows.apply("Validate features", new ValidateTransform(specs)); - pFeatureRows = - PFeatureRows.of( - pFeatureRows - .getMain() - .apply( - "Round event timestamps to granularity", - ParDo.of(new RoundEventTimestampsDoFn())), - pFeatureRows.getErrors()); log.info( "A sample of size 1 of incoming rows from MAIN and ERRORS will logged every 30 seconds for visibility"); logNRows(pFeatureRows, "Output sample", 1, Duration.standardSeconds(30)); - PFeatureRows servingRows = pFeatureRows; - PFeatureRows warehouseRows = pFeatureRows; + PFeatureRows warehouseRows = roundTimestamps("Round timestamps for warehouse", pFeatureRows); + PFeatureRows servingRows = pFeatureRows; if (jobOptions.isCoalesceRowsEnabled()) { // Should we merge and dedupe rows before writing to the serving store? servingRows = servingRows.apply("Coalesce Rows", new CoalescePFeatureRows( jobOptions.getCoalesceRowsDelaySeconds(), jobOptions.getCoalesceRowsTimeoutSeconds())); } + servingRows = roundTimestamps("Round timestamps for serving", servingRows); + if (!dryRun) { servingRows.apply("Write to Serving Stores", servingStoreTransform); warehouseRows.apply("Write to Warehouse Stores", warehouseStoreTransform); @@ -190,6 +184,16 @@ public void expand() { } } + public PFeatureRows roundTimestamps(String name, PFeatureRows pFeatureRows) { + return + PFeatureRows.of( + pFeatureRows + .getMain() + .apply(name, + ParDo.of(new RoundEventTimestampsDoFn())), + pFeatureRows.getErrors()); + } + public PipelineResult run() { PipelineResult result = pipeline.run(); log.info(String.format("FeastImportJobId:%s", this.retrieveId(result))); diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java index 00cc2534d7c..3b02834d778 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java @@ -103,9 +103,11 @@ public static FeatureRow toFeatureRow(CoalesceAccum accum, long counter) { FeatureRow.Builder builder = FeatureRow.newBuilder() .setEntityName(accum.getEntityName()) .setGranularity(accum.getGranularity()) - .setEntityKey(accum.getEntityKey()) - // This will be the latest timestamp - .setEventTimestamp(accum.getEventTimestamp()); + .setEntityKey(accum.getEntityKey()); + // This will be the latest timestamp + if (accum.hasEventTimestamp()) { + builder.setEventTimestamp(accum.getEventTimestamp()); + } Map features = accum.getFeaturesMap(); if (counter <= 0) { @@ -144,13 +146,16 @@ public static CoalesceAccum combineFeatureRows(CoalesceAccum seed, Iterable expand(PInput input) { schema.getFieldsList().size() > 0, "CSV import needs schema with a least one field specified"); + if (!Strings.isNullOrEmpty(timestampColumn)) { + Preconditions.checkArgument(fieldNames.contains(timestampColumn), + String.format("timestampColumn %s, does not match any field", timestampColumn)); + } + PCollection stringMaps = input.getPipeline().apply(CsvIO.read(path, fieldNames)); return stringMaps.apply( @@ -140,7 +145,6 @@ public void processElement(ProcessContext context) { } - public static class CsvFileFeatureSourceOptions implements Options { @NotEmpty diff --git a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java index 38315591efc..259be9a777e 100644 --- a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java +++ b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java @@ -36,6 +36,7 @@ import feast.ingestion.model.Values; import feast.ingestion.options.ImportJobPipelineOptions; import feast.ingestion.service.SpecRetrievalException; +import feast.ingestion.util.DateUtil; import feast.ingestion.util.ProtoUtil; import feast.specs.ImportSpecProto.ImportSpec; import feast.storage.MockErrorsStore; @@ -44,6 +45,7 @@ import feast.storage.service.ErrorsStoreService; import feast.storage.service.ServingStoreService; import feast.storage.service.WarehouseStoreService; +import feast.types.FeatureProto.Feature; import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; import feast.types.GranularityProto.Granularity; @@ -188,16 +190,18 @@ public void testImportCSV_withCoalesceRows() throws IOException { + " - testEntity\n" + "schema:\n" + " entityIdColumn: id\n" - + " timestampValue: 2018-09-25T00:00:00.000Z\n" + + " timestampColumn: timestamp\n" + " fields:\n" + " - name: id\n" + + " - name: timestamp\n" + " - featureId: testEntity.none.testInt32\n" + " - featureId: testEntity.none.testString\n" + "\n", ImportSpec.getDefaultInstance()); File csvFile = folder.newFile("data.csv"); - Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n1,,b\n"); + Files.asCharSink(csvFile, Charsets.UTF_8) + .write("1,2018-09-25T00:00:00.000Z,101,a\n1,2018-09-26T00:00:00.000Z,,b\n"); importSpec = initImportSpec(importSpec, csvFile.toString()); ImportJobPipelineOptions options = initOptions(); @@ -227,22 +231,21 @@ public void testImportCSV_withCoalesceRows() throws IOException { PAssert.that(writtenToErrors).satisfies(hasCount(0)); PAssert.that(writtenToServing.apply("serving toFeatureRows", new ToOrderedFeatureRows())) - .containsInAnyOrder(normalize( - FeatureRow.newBuilder() - .setGranularity(Granularity.Enum.NONE) - .setEventTimestamp(Timestamp.getDefaultInstance()) - .setEntityKey("1") - .setEntityName("testEntity") - .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) - .addFeatures(Features.of("testEntity.none.testString", Values.ofString("b"))) - .build())); + .containsInAnyOrder( + normalize( + FeatureRow.newBuilder() + .setGranularity(Granularity.Enum.NONE) + .setEntityKey("1") + .setEntityName("testEntity") + .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) + .addFeatures(Features.of("testEntity.none.testString", Values.ofString("b"))) + .build())); PAssert.that(writtenToWarehouse.apply("warehouse toFeatureRows", new ToOrderedFeatureRows())) .containsInAnyOrder( normalize( FeatureRow.newBuilder() .setGranularity(Granularity.Enum.NONE) - .setEventTimestamp(Timestamp.getDefaultInstance()) .setEntityKey("1") .setEntityName("testEntity") .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) @@ -251,7 +254,6 @@ public void testImportCSV_withCoalesceRows() throws IOException { normalize( FeatureRow.newBuilder() .setGranularity(Granularity.Enum.NONE) - .setEventTimestamp(Timestamp.getDefaultInstance()) .setEntityKey("1") .setEntityName("testEntity") .addFeatures(Features.of("testEntity.none.testString", Values.ofString("b"))) diff --git a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java index 11dba243c17..4d46e8925a9 100644 --- a/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java +++ b/ingestion/src/test/java/feast/ingestion/transform/CoalesceFeatureRowsTest.java @@ -149,9 +149,11 @@ public void testBatch_shouldMergeFeatures() { public void testStream_shouldMergeFeatures() { List rows = Lists.newArrayList( FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(DEFAULT_TIMESTAMP) .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .build(), FeatureRow.newBuilder().setEntityKey("1") + .setEventTimestamp(DEFAULT_TIMESTAMP) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) .build()); @@ -169,7 +171,7 @@ public void testStream_shouldMergeFeatures() { PAssert.that(output.apply(Count.globally())).containsInAnyOrder(1L); PAssert.that(output).containsInAnyOrder( FeatureRow.newBuilder().setEntityKey("1") - .setEventTimestamp(Timestamp.getDefaultInstance()) + .setEventTimestamp(DEFAULT_TIMESTAMP) .addFeatures(Feature.newBuilder().setId("f1").setValue(Values.ofInt32(1))) .addFeatures(Feature.newBuilder().setId("f2").setValue(Values.ofInt32(2))) .build() @@ -261,10 +263,10 @@ public void testStream_shouldNotSetTimerWhilePending() { Duration delay = Duration.standardSeconds(10); TestStream testStream = TestStream.create(ProtoCoder.of(FeatureRow.class)) .advanceWatermarkTo(start) - .addElements(FeatureRow.getDefaultInstance()) + .addElements(DEFAULT_FEATURE_ROW) // this should not reset the timer as the first is still pending. .advanceWatermarkTo(start.plus(delay.dividedBy(2))) - .addElements(FeatureRow.getDefaultInstance()) + .addElements(DEFAULT_FEATURE_ROW) // timer should trigger causing the first output row .advanceWatermarkTo(start.plus(delay).plus(delay.dividedBy(2))) .addElements(FeatureRow.getDefaultInstance()) // this should cause a second output row. @@ -287,12 +289,14 @@ public void testStream_shouldOnlyEmitNewFeaturesInSecondPane() { .advanceWatermarkTo(start) .addElements( FeatureRow.newBuilder() + .setEventTimestamp(DEFAULT_TIMESTAMP) .addFeatures(Features.of("f1", Values.ofString("a"))) .build()) // this should should emit a row .advanceWatermarkTo(start.plus(delay).plus(delay)) .addElements( FeatureRow.newBuilder() + .setEventTimestamp(DEFAULT_TIMESTAMP) .addFeatures(Features.of("f2", Values.ofString("b"))) .build()) // this should emit a row with f2 but without f1 because it hasn't had an update From 07104d43133f0137aacce3fd6514bc4d23dd600f Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Thu, 24 Jan 2019 20:22:07 +0700 Subject: [PATCH 05/15] rename importSpec.options to be importspec.sourceOptions and add jobOptions --- core/src/main/java/feast/core/model/JobInfo.java | 11 ++++++++--- .../feast/core/validators/SpecValidator.java | 16 ++++++++-------- .../feast/core/job/ScheduledJobMonitorTest.java | 2 ++ .../test/java/feast/core/model/JobInfoTest.java | 11 +++++------ .../core/service/JobManagementServiceTest.java | 3 +++ .../feast/core/validators/SpecValidatorTest.java | 16 ++++++++-------- 6 files changed, 34 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/feast/core/model/JobInfo.java b/core/src/main/java/feast/core/model/JobInfo.java index 3f59445bf39..e45f7b66372 100644 --- a/core/src/main/java/feast/core/model/JobInfo.java +++ b/core/src/main/java/feast/core/model/JobInfo.java @@ -55,8 +55,12 @@ public class JobInfo extends AbstractTimestampEntity { private String runner; // Job options. Stored as a json string as it is specific to the runner. - @Column(name = "options") - private String options; + @Column(name = "source_options") + private String sourceOptions; + + // Job options. Stored as a json string as it is specific to the runner. + @Column(name = "job_options") + private String jobOptions; // Entities populated by the job @ManyToMany @@ -99,7 +103,8 @@ public JobInfo( this.extId = extId; this.type = importSpec.getType(); this.runner = runner; - this.options = TypeConversion.convertMapToJsonString(importSpec.getOptionsMap()); + this.sourceOptions = TypeConversion.convertMapToJsonString(importSpec.getSourceOptionsMap()); + this.jobOptions = TypeConversion.convertMapToJsonString(importSpec.getJobOptionsMap()); this.entities = new ArrayList<>(); for (String entity : importSpec.getEntitiesList()) { EntityInfo entityInfo = new EntityInfo(); diff --git a/core/src/main/java/feast/core/validators/SpecValidator.java b/core/src/main/java/feast/core/validators/SpecValidator.java index c8500ac331b..8259607e5e4 100644 --- a/core/src/main/java/feast/core/validators/SpecValidator.java +++ b/core/src/main/java/feast/core/validators/SpecValidator.java @@ -264,8 +264,8 @@ public void validateImportSpec(ImportSpec spec) throws IllegalArgumentException private void checkKafkaImportSpecOption(ImportSpec spec) { try { - String topics = spec.getOptionsOrDefault("topics", ""); - String server = spec.getOptionsOrDefault("server", ""); + String topics = spec.getSourceOptionsOrDefault("topics", ""); + String server = spec.getSourceOptionsOrDefault("server", ""); if (topics.equals("") && server.equals("")) { throw new IllegalArgumentException( "Kafka ingestion requires either topics or servers"); @@ -278,7 +278,7 @@ private void checkKafkaImportSpecOption(ImportSpec spec) { private void checkFileImportSpecOption(ImportSpec spec) throws IllegalArgumentException { try { - checkArgument(!spec.getOptionsOrDefault("path", "").equals(""), "File path cannot be empty"); + checkArgument(!spec.getSourceOptionsOrDefault("path", "").equals(""), "File path cannot be empty"); } catch (NullPointerException | IllegalArgumentException e) { throw new IllegalArgumentException( Strings.lenientFormat("Invalid options: %s", e.getMessage())); @@ -287,8 +287,8 @@ private void checkFileImportSpecOption(ImportSpec spec) throws IllegalArgumentEx private void checkPubSubImportSpecOption(ImportSpec spec) throws IllegalArgumentException { try { - String topic = spec.getOptionsOrDefault("topic", ""); - String subscription = spec.getOptionsOrDefault("subscription", ""); + String topic = spec.getSourceOptionsOrDefault("topic", ""); + String subscription = spec.getSourceOptionsOrDefault("subscription", ""); if (topic.equals("") && subscription.equals("")) { throw new IllegalArgumentException( "Pubsub ingestion requires either topic or subscription"); @@ -301,11 +301,11 @@ private void checkPubSubImportSpecOption(ImportSpec spec) throws IllegalArgument private void checkBigqueryImportSpecOption(ImportSpec spec) throws IllegalArgumentException { try { - checkArgument(!spec.getOptionsOrThrow("project").equals(""), + checkArgument(!spec.getSourceOptionsOrThrow("project").equals(""), "Bigquery project cannot be empty"); - checkArgument(!spec.getOptionsOrThrow("dataset").equals(""), + checkArgument(!spec.getSourceOptionsOrThrow("dataset").equals(""), "Bigquery dataset cannot be empty"); - checkArgument(!spec.getOptionsOrThrow("table").equals(""), "Bigquery table cannot be empty"); + checkArgument(!spec.getSourceOptionsOrThrow("table").equals(""), "Bigquery table cannot be empty"); } catch (NullPointerException | IllegalArgumentException e) { throw new IllegalArgumentException( Strings.lenientFormat("Invalid options: %s", e.getMessage())); diff --git a/core/src/test/java/feast/core/job/ScheduledJobMonitorTest.java b/core/src/test/java/feast/core/job/ScheduledJobMonitorTest.java index d352044d16e..1a05869657f 100644 --- a/core/src/test/java/feast/core/job/ScheduledJobMonitorTest.java +++ b/core/src/test/java/feast/core/job/ScheduledJobMonitorTest.java @@ -66,6 +66,7 @@ public void getJobStatus_shouldUpdateJobInfoForRunningJob() { "Streaming", "DataflowRunner", "", + "", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), @@ -104,6 +105,7 @@ public void getJobMetrics_shouldPushToStatsDMetricPusherAndSaveNewMetricToDb() { "Streaming", "DataflowRunner", "", + "", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), diff --git a/core/src/test/java/feast/core/model/JobInfoTest.java b/core/src/test/java/feast/core/model/JobInfoTest.java index a75050d6022..f47d8435f83 100644 --- a/core/src/test/java/feast/core/model/JobInfoTest.java +++ b/core/src/test/java/feast/core/model/JobInfoTest.java @@ -41,9 +41,8 @@ public void shouldInitialiseGivenJobIdAndSpec() throws InvalidProtocolBufferExce .build(); ImportSpecProto.ImportSpec importSpec = ImportSpecProto.ImportSpec.newBuilder() - .setType("file") - .putOptions("format", "csv") - .putOptions("path", "gs://some/path") + .setType("file.csv") + .putSourceOptions("path", "gs://some/path") .addEntities("entity") .setSchema(schema) .build(); @@ -52,9 +51,9 @@ public void shouldInitialiseGivenJobIdAndSpec() throws InvalidProtocolBufferExce JobInfo expected = new JobInfo(); expected.setId("fake-job-id"); expected.setExtId("fake-ext-id"); - expected.setType("file"); + expected.setType("file.csv"); expected.setRunner("DataflowRunner"); - expected.setOptions(TypeConversion.convertMapToJsonString(importSpec.getOptionsMap())); + expected.setSourceOptions(TypeConversion.convertMapToJsonString(importSpec.getSourceOptionsMap())); List entities = new ArrayList<>(); EntityInfo entityInfo = new EntityInfo(); @@ -75,7 +74,7 @@ public void shouldInitialiseGivenJobIdAndSpec() throws InvalidProtocolBufferExce assertThat(actual.getRunner(), equalTo(expected.getRunner())); assertThat(actual.getEntities(), equalTo(expected.getEntities())); assertThat(actual.getFeatures(), equalTo(expected.getFeatures())); - assertThat(actual.getOptions(), equalTo(expected.getOptions())); + assertThat(actual.getSourceOptions(), equalTo(expected.getSourceOptions())); assertThat(actual.getRaw(), equalTo(expected.getRaw())); } } \ No newline at end of file diff --git a/core/src/test/java/feast/core/service/JobManagementServiceTest.java b/core/src/test/java/feast/core/service/JobManagementServiceTest.java index 709cf365b77..1902ccc578d 100644 --- a/core/src/test/java/feast/core/service/JobManagementServiceTest.java +++ b/core/src/test/java/feast/core/service/JobManagementServiceTest.java @@ -70,6 +70,7 @@ public void shouldListAllJobDetails() { "", "", "", + "", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), @@ -84,6 +85,7 @@ public void shouldListAllJobDetails() { "", "", "", + "", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), @@ -121,6 +123,7 @@ public void shouldReturnDetailOfRequestedJobId() { "", "", "", + "", Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), diff --git a/core/src/test/java/feast/core/validators/SpecValidatorTest.java b/core/src/test/java/feast/core/validators/SpecValidatorTest.java index 0c666a11a6d..3cf2caf6782 100644 --- a/core/src/test/java/feast/core/validators/SpecValidatorTest.java +++ b/core/src/test/java/feast/core/validators/SpecValidatorTest.java @@ -709,7 +709,7 @@ public void fileImportSpecWithoutEntityIdColumnInSchemaShouldThrowIllegalArgumen ImportSpec input = ImportSpec.newBuilder() .setType("file.csv") - .putOptions("path", "gs://asdasd") + .putSourceOptions("path", "gs://asdasd") .build(); exception.expect(IllegalArgumentException.class); exception.expectMessage( @@ -728,9 +728,9 @@ public void bigQueryImportSpecWithoutEntityIdColumnInSchemaShouldThrowIllegalArg ImportSpec input = ImportSpec.newBuilder() .setType("bigquery") - .putOptions("project", "my-google-project") - .putOptions("dataset", "feast") - .putOptions("table", "feast") + .putSourceOptions("project", "my-google-project") + .putSourceOptions("dataset", "feast") + .putSourceOptions("table", "feast") .build(); exception.expect(IllegalArgumentException.class); exception.expectMessage( @@ -750,7 +750,7 @@ public void importSpecWithoutValidEntityShouldThrowIllegalArgumentException() { ImportSpec input = ImportSpec.newBuilder() .setType("pubsub") - .putOptions("topic", "my/pubsub/topic") + .putSourceOptions("topic", "my/pubsub/topic") .addEntities("someEntity") .build(); exception.expect(IllegalArgumentException.class); @@ -775,7 +775,7 @@ public void importSpecWithUnregisteredFeaturesShouldThrowIllegalArgumentExceptio ImportSpec input = ImportSpec.newBuilder() .setType("pubsub") - .putOptions("topic", "my/pubsub/topic") + .putSourceOptions("topic", "my/pubsub/topic") .setSchema(schema) .addEntities("someEntity") .build(); @@ -802,8 +802,8 @@ public void importSpecWithKafkaSourceAndCorrectOptionsShouldPassValidation() { ImportSpec input = ImportSpec.newBuilder() .setType("kafka") - .putOptions("topics", "my-kafka-topic") - .putOptions("server", "localhost:54321") + .putSourceOptions("topics", "my-kafka-topic") + .putSourceOptions("server", "localhost:54321") .setSchema(schema) .addEntities("someEntity") .build(); From 581762ee7b9b0bfb58e19f74b507106710da8b07 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Sun, 27 Jan 2019 22:51:42 +0800 Subject: [PATCH 06/15] only write out the latest row for bigtable and remove postgres feature store --- ingestion/pom.xml | 12 -- .../main/java/feast/ingestion/ImportJob.java | 1 - .../options/ImportJobPipelineOptions.java | 11 +- .../feast/ingestion/options/JobOptions.java | 6 + .../bigtable/FeatureRowBigTableIO.java | 19 +- .../FeatureRowToBigTableMutationDoFn.java | 63 ++---- .../storage/postgres/FeatureRowJdbcIO.java | 199 ------------------ .../storage/postgres/PostgresOptions.java | 38 ---- .../postgres/PostgresServingStore.java | 47 ----- .../feast/ingestion/ImportJobCSVTest.java | 74 ++++++- .../storage/postgres/PostgresOptionsTest.java | 137 ------------ 11 files changed, 100 insertions(+), 507 deletions(-) delete mode 100644 ingestion/src/main/java/feast/storage/postgres/FeatureRowJdbcIO.java delete mode 100644 ingestion/src/main/java/feast/storage/postgres/PostgresOptions.java delete mode 100644 ingestion/src/main/java/feast/storage/postgres/PostgresServingStore.java delete mode 100644 ingestion/src/test/java/feast/storage/postgres/PostgresOptionsTest.java diff --git a/ingestion/pom.xml b/ingestion/pom.xml index 69823921a9c..3abce057dd1 100644 --- a/ingestion/pom.xml +++ b/ingestion/pom.xml @@ -226,12 +226,6 @@ ${org.apache.beam.version} - - org.apache.beam - beam-sdks-java-io-jdbc - ${org.apache.beam.version} - - org.apache.beam beam-sdks-java-io-kafka @@ -302,12 +296,6 @@ 5.87.0.RELEASE - - org.postgresql - postgresql - 42.2.5 - - com.github.kstyrc embedded-redis diff --git a/ingestion/src/main/java/feast/ingestion/ImportJob.java b/ingestion/src/main/java/feast/ingestion/ImportJob.java index 56977cf76e4..32420698253 100644 --- a/ingestion/src/main/java/feast/ingestion/ImportJob.java +++ b/ingestion/src/main/java/feast/ingestion/ImportJob.java @@ -167,7 +167,6 @@ public void expand() { logNRows(pFeatureRows, "Output sample", 1, Duration.standardSeconds(30)); PFeatureRows warehouseRows = roundTimestamps("Round timestamps for warehouse", pFeatureRows); - PFeatureRows servingRows = pFeatureRows; if (jobOptions.isCoalesceRowsEnabled()) { // Should we merge and dedupe rows before writing to the serving store? diff --git a/ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java b/ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java index 2ec4fa44798..72e8b9a53a1 100644 --- a/ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java +++ b/ingestion/src/main/java/feast/ingestion/options/ImportJobPipelineOptions.java @@ -26,6 +26,9 @@ import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.options.Validation.Required; +/** + * Options passed to Beam to influence the job's execution environment + */ public interface ImportJobPipelineOptions extends PipelineOptions { @Description("Import spec yaml file path") @@ -59,14 +62,6 @@ public interface ImportJobPipelineOptions extends PipelineOptions { void setCoreApiSpecPath(String value); - @Override - @Description("The beam sink class to which the metrics will be pushed") - @Default.InstanceFactory(NoOpMetricsSink.class) - Class getMetricsSink(); - - @Override - void setMetricsSink(Class metricsSink); - @Description( "Set an errors store type. One of: [stderr, stdout, file.json]. Note that you should not use " + "stderr/stdout in production unless your data volume is extremely small.") diff --git a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java index 8cbe9ade7a9..68017644e82 100644 --- a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java +++ b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java @@ -20,6 +20,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import feast.options.Options; +/** + * JobOptions are options passed in via the import spec, they are options that dictate certain + * behaviour of the job, they differ from the PipelineOptions in ImportJobOptions which are used to + * influence the execution environment. + */ public class JobOptions implements Options { private long sampleLimit; @@ -27,6 +32,7 @@ public class JobOptions implements Options { private long coalesceRowsDelaySeconds; private long coalesceRowsTimeoutSeconds; + @JsonProperty(value = "sample.limit") public long getSampleLimit() { return sampleLimit; diff --git a/ingestion/src/main/java/feast/storage/bigtable/FeatureRowBigTableIO.java b/ingestion/src/main/java/feast/storage/bigtable/FeatureRowBigTableIO.java index 67d1c2181c8..7437f59c55a 100644 --- a/ingestion/src/main/java/feast/storage/bigtable/FeatureRowBigTableIO.java +++ b/ingestion/src/main/java/feast/storage/bigtable/FeatureRowBigTableIO.java @@ -19,20 +19,17 @@ import com.google.cloud.bigtable.beam.CloudBigtableConfiguration; import com.google.cloud.bigtable.beam.CloudBigtableIO; +import feast.ingestion.model.Specs; +import feast.ingestion.transform.FeatureIO; +import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import java.util.Collections; import lombok.extern.slf4j.Slf4j; -import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.transforms.DoFn; 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.PDone; import org.apache.hadoop.hbase.client.Mutation; -import feast.ingestion.model.Specs; -import feast.ingestion.transform.FeatureIO; -import feast.ingestion.transform.SplitFeatures.SingleOutputSplit; -import feast.specs.FeatureSpecProto.FeatureSpec; -import feast.types.FeatureRowExtendedProto.FeatureRowExtended; @Slf4j public class FeatureRowBigTableIO { @@ -51,17 +48,9 @@ public Write(BigTableStoreOptions bigTableOptions, Specs specs) { public PDone expand(PCollection input) { log.info("Using BigTable options: " + bigTableOptions.toString()); - // we need a row per granularity, because they will have different keys in BigTable - PCollection features = - input - .apply( - "Split by granularity", - new SingleOutputSplit<>(FeatureSpec::getGranularity, specs)) - .setCoder(ProtoCoder.of(FeatureRowExtended.class)); - // entity name to mutation key value PCollection> mutations = - features.apply( + input.apply( "Map to BigTable mutations", ParDo.of(new FeatureRowToBigTableMutationDoFn(bigTableOptions.prefix, specs))); diff --git a/ingestion/src/main/java/feast/storage/bigtable/FeatureRowToBigTableMutationDoFn.java b/ingestion/src/main/java/feast/storage/bigtable/FeatureRowToBigTableMutationDoFn.java index b36f84dcdda..bfd44ceda87 100644 --- a/ingestion/src/main/java/feast/storage/bigtable/FeatureRowToBigTableMutationDoFn.java +++ b/ingestion/src/main/java/feast/storage/bigtable/FeatureRowToBigTableMutationDoFn.java @@ -18,26 +18,21 @@ package feast.storage.bigtable; import com.google.common.base.Charsets; -import com.google.common.collect.Lists; -import java.util.List; -import lombok.extern.slf4j.Slf4j; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.values.KV; -import org.apache.commons.codec.digest.DigestUtils; -import org.apache.hadoop.hbase.client.Mutation; -import org.apache.hadoop.hbase.client.Put; import feast.SerializableCache; import feast.ingestion.model.Specs; import feast.ingestion.util.DateUtil; import feast.options.OptionsParser; -import feast.specs.EntitySpecProto.EntitySpec; import feast.specs.FeatureSpecProto.FeatureSpec; import feast.storage.BigTableProto.BigTableRowKey; import feast.types.FeatureProto.Feature; import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; -import feast.types.GranularityProto.Granularity; -import feast.types.GranularityProto.Granularity.Enum; +import lombok.extern.slf4j.Slf4j; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.commons.codec.digest.DigestUtils; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Put; /** * DoFn for taking a feature row and making Bigtable mutations out of it. Also keys the mutations by @@ -47,6 +42,8 @@ public class FeatureRowToBigTableMutationDoFn extends DoFn> { + private static final String LATEST_KEY = "0"; + private final SerializableCache servingOptionsCache = SerializableCache.builder() .loadingFunction( @@ -64,33 +61,21 @@ public class FeatureRowToBigTableMutationDoFn } public static BigTableRowKey makeBigTableRowKey( - String entityKey, com.google.protobuf.Timestamp timestamp, Granularity.Enum granularity) { + String entityKey) { return BigTableRowKey.newBuilder() .setSha1Prefix(DigestUtils.sha1Hex(entityKey).substring(0, 7)) .setEntityKey(entityKey) - .setReversedMillis(String.valueOf(getReversedRoundedMillis(timestamp, granularity))) + .setReversedMillis(LATEST_KEY) .build(); } - public static long getReversedRoundedMillis( - com.google.protobuf.Timestamp timestamp, Granularity.Enum granularity) { - if (granularity == Granularity.Enum.NONE) { - return 0L; // We store it as zero instead of reversed Long.MAX_VALUE. - } - timestamp = DateUtil.roundToGranularity(timestamp, granularity); - return Long.MAX_VALUE - timestamp.getSeconds() * 1000; - } - @ProcessElement public void processElement(ProcessContext context) { FeatureRowExtended rowExtended = context.element(); FeatureRow row = rowExtended.getRow(); - EntitySpec entitySpec = specs.getEntitySpec(row.getEntityName()); - List mutations = makePut(rowExtended); - for (Put put : mutations) { - context.output(KV.of(getTableName(row), put)); - } + Put put = makePut(rowExtended); + context.output(KV.of(getTableName(row), put)); } private String getTableName(FeatureRow row) { @@ -107,21 +92,12 @@ private String getTableName(FeatureRow row) { *

bigtable row key = {sha1(row.key), row.key, row.timestamp} family = {feature.group, * granularity} qualifier = {feature.name} value = {feature.value} */ - public List makePut(FeatureRowExtended rowExtended) { + public Put makePut(FeatureRowExtended rowExtended) { FeatureRow row = rowExtended.getRow(); - Granularity.Enum granularity = row.getGranularity(); // We always additinally overwrite a None granularity row so that it is trivial to retrieve the // latest across all features. Put latestPut = - new Put( - makeBigTableRowKey(row.getEntityKey(), row.getEventTimestamp(), Enum.NONE) - .toByteArray()); - Put timeseriesPut = - new Put( - makeBigTableRowKey(row.getEntityKey(), row.getEventTimestamp(), granularity) - .toByteArray()); - boolean isTimeseries = granularity.getNumber() != Enum.NONE.getNumber(); - + new Put(makeBigTableRowKey(row.getEntityKey()).toByteArray()); for (Feature feature : row.getFeaturesList()) { FeatureSpec featureSpec = specs.getFeatureSpec(feature.getId()); BigTableFeatureOptions options = servingOptionsCache.get(featureSpec); @@ -129,18 +105,9 @@ public List makePut(FeatureRowExtended rowExtended) { byte[] family = options.family.getBytes(Charsets.UTF_8); byte[] qualifier = feature.getId().getBytes(Charsets.UTF_8); byte[] value = feature.getValue().toByteArray(); - // Note version will always the same for row key. long version = DateUtil.toMillis(row.getEventTimestamp()); latestPut.addColumn(family, qualifier, version, value); - if (isTimeseries) { - timeseriesPut.addColumn(family, qualifier, version, value); - } - } - - if (isTimeseries) { - return Lists.newArrayList(timeseriesPut, latestPut); - } else { - return Lists.newArrayList(latestPut); } + return latestPut; } } diff --git a/ingestion/src/main/java/feast/storage/postgres/FeatureRowJdbcIO.java b/ingestion/src/main/java/feast/storage/postgres/FeatureRowJdbcIO.java deleted file mode 100644 index 960b21056d5..00000000000 --- a/ingestion/src/main/java/feast/storage/postgres/FeatureRowJdbcIO.java +++ /dev/null @@ -1,199 +0,0 @@ -/* - * 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.storage.postgres; - -import com.google.common.base.Charsets; -import com.google.common.base.Strings; -import com.google.common.collect.Lists; -import feast.ingestion.util.DateUtil; -import feast.ingestion.transform.FeatureIO; -import feast.ingestion.transform.SplitFeatures.MultiOutputSplit; -import feast.ingestion.model.Specs; -import com.google.inject.Inject; -import feast.specs.FeatureSpecProto.FeatureSpec; -import feast.types.GranularityProto.Granularity; -import feast.types.FeatureProto.Feature; -import feast.types.FeatureRowExtendedProto.FeatureRowExtended; -import feast.types.FeatureRowProto.FeatureRow; -import feast.types.ValueProto.Value; -import java.io.ByteArrayOutputStream; -import java.sql.PreparedStatement; -import java.sql.SQLException; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; -import lombok.extern.slf4j.Slf4j; -import org.apache.beam.sdk.io.jdbc.JdbcIO; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.TupleTag; -import org.jtwig.JtwigModel; -import org.jtwig.JtwigTemplate; - -@Slf4j -public class FeatureRowJdbcIO { - - - - public static class Write extends FeatureIO.Write { - - private String driverClassName = "org.postgresql.Driver"; - private PostgresOptions jdbcOptions; - private Specs specs; - private String storeId; - - private static Map driverTemplates = new HashMap<>(); - - static { - driverTemplates.put("org.postgresql.Driver", "upsert_feature_row.postgres.sql.twig"); - } - - @Inject - public Write(PostgresOptions jdbcOptions, Specs specs, String storeId) { - this.jdbcOptions = jdbcOptions; - this.specs = specs; - this.storeId = storeId; - } - - public static String getTableName(FeatureSpec spec) { - String entity = spec.getEntity(); - String granularity = spec.getGranularity().name().toLowerCase(); - return String.format("%s_%s", entity, granularity); - } - - @Override - public PDone expand(PCollection input) { - List storeFeatureSpecs = specs.getFeatureSpecByServingStoreId(storeId); - Set tableNames = new HashSet<>(); - Map> featuresByTable = new HashMap<>(); - - // Group feature specs by table name - for (FeatureSpec featureSpec : storeFeatureSpecs) { - String tableName = getTableName(featureSpec); - tableNames.add(tableName); - Set featureSpecs = featuresByTable.getOrDefault(tableName, new HashSet<>()); - featureSpecs.add(featureSpec); - featuresByTable.put(tableName, featureSpecs); - } - - MultiOutputSplit splitter = - new MultiOutputSplit<>(Write::getTableName, tableNames, specs); - PCollectionTuple splitByTable = input.apply(splitter); - - for (String tableName : tableNames) { - log.info("Initialising write transform for table " + tableName); - TupleTag tag = splitter.getStrategy().getTag(tableName); - writeTable(splitByTable.get(tag), Lists.newArrayList(featuresByTable.get(tableName))); - } - return PDone.in(input.getPipeline()); - } - - private PDone writeTable( - PCollection input, List featureSpecs) { - String tableName = - Strings.nullToEmpty(jdbcOptions.prefix) + getTableName(featureSpecs.get(0)); - - // We create map of positions so that the prepared statement knows what order to set feature - // columns - final Map positions = new HashMap<>(); - for (int i = 0; i < featureSpecs.size(); i++) { - positions.put(featureSpecs.get(i).getId(), i); - } - - JtwigTemplate template = - JtwigTemplate.classpathTemplate("templates/" + driverTemplates.get(driverClassName)); - JtwigModel model = - JtwigModel.newModel() - .with("tableName", tableName) - .with( - "featureNames", - featureSpecs.stream().map(FeatureSpec::getName).collect(Collectors.toList())); - - ByteArrayOutputStream outs = new ByteArrayOutputStream(); - template.render(model, outs); - String statement = new String(outs.toByteArray(), Charsets.UTF_8); - - return input.apply( - JdbcIO.write() - .withDataSourceConfiguration( - JdbcIO.DataSourceConfiguration.create(driverClassName, jdbcOptions.url)) - .withStatement(statement) - .withPreparedStatementSetter( - new JdbcIO.PreparedStatementSetter() { - public void setParameters( - FeatureRowExtended extendedRow, PreparedStatement query) - throws SQLException { - // We assume all features have the same granularity - // (this should be enforced by validation) - - FeatureRow row = extendedRow.getRow(); - Granularity.Enum granularity = row.getGranularity(); - - query.setString(1, row.getEntityKey()); - query.setTimestamp(2, DateUtil.toSqlTimestamp(row.getEventTimestamp())); - - int featuresOffset = 3; - for (Feature feature : row.getFeaturesList()) { - Integer featurePosition = positions.get(feature.getId()); - if (featurePosition == null) { - continue; - } - setPreparedStatementValue( - query, featuresOffset + featurePosition, feature.getValue()); - } - } - })); - } - - private void setPreparedStatementValue(PreparedStatement query, int index, Value value) - throws SQLException { - switch (value.getValCase()) { - case STRINGVAL: - query.setString(index, value.getStringVal()); - break; - case TIMESTAMPVAL: - query.setTimestamp(index, DateUtil.toSqlTimestamp(value.getTimestampVal())); - break; - case DOUBLEVAL: - query.setDouble(index, value.getDoubleVal()); - break; - case BOOLVAL: - query.setBoolean(index, value.getBoolVal()); - break; - case BYTESVAL: - query.setBytes(index, value.getBytesVal().toByteArray()); - break; - case FLOATVAL: - query.setFloat(index, value.getFloatVal()); - break; - case INT64VAL: - query.setLong(index, value.getInt64Val()); - break; - case INT32VAL: - query.setInt(index, value.getInt32Val()); - break; - default: - throw new SQLException("Unhandled type " + value.getValCase().name()); - } - } - } -} diff --git a/ingestion/src/main/java/feast/storage/postgres/PostgresOptions.java b/ingestion/src/main/java/feast/storage/postgres/PostgresOptions.java deleted file mode 100644 index 0d6dda9e31b..00000000000 --- a/ingestion/src/main/java/feast/storage/postgres/PostgresOptions.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * 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.storage.postgres; - -import com.fasterxml.jackson.annotation.JsonProperty; -import java.io.Serializable; -import javax.validation.constraints.NotNull; -import javax.validation.constraints.Pattern; -import feast.options.Options; - -public class PostgresOptions implements Options, Serializable { - private static final String OPT_POSTGRES_URL = "postgres.url"; - private static final String OPT_POSTGRES_PREFIX = "postgres.prefix"; - - @JsonProperty(value = OPT_POSTGRES_URL) - @NotNull - @Pattern(regexp = "jdbc:postgresql:.+") - String url; - - @JsonProperty(value = OPT_POSTGRES_PREFIX) - @Pattern(regexp = "[a-zA-Z_][a-zA-Z0-9_]*") - String prefix; -} diff --git a/ingestion/src/main/java/feast/storage/postgres/PostgresServingStore.java b/ingestion/src/main/java/feast/storage/postgres/PostgresServingStore.java deleted file mode 100644 index 9f68833b2fc..00000000000 --- a/ingestion/src/main/java/feast/storage/postgres/PostgresServingStore.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * 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.storage.postgres; - -import com.google.auto.service.AutoService; -import com.google.common.base.Preconditions; -import feast.ingestion.model.Specs; -import feast.ingestion.transform.FeatureIO.Write; -import feast.options.OptionsParser; -import feast.specs.StorageSpecProto.StorageSpec; -import feast.storage.ServingStore; - -@AutoService(ServingStore.class) -public class PostgresServingStore implements ServingStore { - public static String TYPE_POSTGRES = "postgres"; - - @Override - public Write create(StorageSpec storageSpec, Specs specs) { - Preconditions.checkArgument( - storageSpec.getType().equals(TYPE_POSTGRES), "Storage spec type was not " + TYPE_POSTGRES); - - PostgresOptions options = - OptionsParser.parse(storageSpec.getOptionsMap(), PostgresOptions.class); - - return new FeatureRowJdbcIO.Write(options, specs, storageSpec.getId()); - } - - @Override - public String getType() { - return TYPE_POSTGRES; - } -} diff --git a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java index 259be9a777e..8dc238b8a3a 100644 --- a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java +++ b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java @@ -36,7 +36,6 @@ import feast.ingestion.model.Values; import feast.ingestion.options.ImportJobPipelineOptions; import feast.ingestion.service.SpecRetrievalException; -import feast.ingestion.util.DateUtil; import feast.ingestion.util.ProtoUtil; import feast.specs.ImportSpecProto.ImportSpec; import feast.storage.MockErrorsStore; @@ -45,7 +44,6 @@ import feast.storage.service.ErrorsStoreService; import feast.storage.service.ServingStoreService; import feast.storage.service.WarehouseStoreService; -import feast.types.FeatureProto.Feature; import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; import feast.types.GranularityProto.Granularity; @@ -176,6 +174,78 @@ public void testImportCSV() throws IOException { testPipeline.run(); } + @Test + public void testImportCSV_withSample1() throws IOException { + ImportSpec importSpec = + ProtoUtil.decodeProtoYaml( + "---\n" + + "type: file.csv\n" + + "sourceOptions:\n" + + " # path: to be overwritten in tests\n" + + "jobOptions:\n" + + " sample.limit: 1\n" + + "entities:\n" + + " - testEntity\n" + + "schema:\n" + + " entityIdColumn: id\n" + + " timestampValue: 2018-09-25T00:00:00.000Z\n" + + " fields:\n" + + " - name: id\n" + + " - featureId: testEntity.none.testInt32\n" + + " - featureId: testEntity.none.testString\n" + + "\n", + ImportSpec.getDefaultInstance()); + + File csvFile = folder.newFile("data.csv"); + Files.asCharSink(csvFile, Charsets.UTF_8).write("1,101,a\n2,202,b\n3,303,c\n"); + importSpec = initImportSpec(importSpec, csvFile.toString()); + + ImportJobPipelineOptions options = initOptions(); + options.setErrorsStoreType(MOCK_ERRORS_STORE_TYPE); + + Injector injector = + Guice.createInjector( + new ImportJobModule(options, importSpec), new TestPipelineModule(testPipeline)); + + ImportJob job = injector.getInstance(ImportJob.class); + injector.getInstance(ImportJob.class); + job.expand(); + + PCollection writtenToServing = + PCollectionList.of(ServingStoreService.get(MockServingStore.class).getWrite().getInputs()) + .apply("flatten serving input", Flatten.pCollections()); + + PCollection writtenToWarehouse = + PCollectionList.of( + WarehouseStoreService.get(MockWarehouseStore.class).getWrite().getInputs()) + .apply("flatten warehouse input", Flatten.pCollections()); + + PCollection writtenToErrors = + PCollectionList.of(ErrorsStoreService.get(MockErrorsStore.class).getWrite().getInputs()) + .apply("flatten errors input", Flatten.pCollections()); + + List expectedRows = + Lists.newArrayList( + normalize( + FeatureRow.newBuilder() + .setGranularity(Granularity.Enum.NONE) + .setEventTimestamp(Timestamp.getDefaultInstance()) + .setEntityKey("1") + .setEntityName("testEntity") + .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) + .addFeatures(Features.of("testEntity.none.testString", Values.ofString("a"))) + .build())); + + PAssert.that(writtenToErrors).satisfies(hasCount(0)); + + PAssert.that(writtenToServing.apply("serving toFeatureRows", new ToOrderedFeatureRows())) + .containsInAnyOrder(expectedRows); + + PAssert.that(writtenToWarehouse.apply("warehouse toFeatureRows", new ToOrderedFeatureRows())) + .containsInAnyOrder(expectedRows); + testPipeline.run(); + } + @Test public void testImportCSV_withCoalesceRows() throws IOException { ImportSpec importSpec = diff --git a/ingestion/src/test/java/feast/storage/postgres/PostgresOptionsTest.java b/ingestion/src/test/java/feast/storage/postgres/PostgresOptionsTest.java deleted file mode 100644 index f5f53de26f5..00000000000 --- a/ingestion/src/test/java/feast/storage/postgres/PostgresOptionsTest.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.storage.postgres; - -import static junit.framework.TestCase.fail; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import lombok.extern.slf4j.Slf4j; -import org.junit.Test; -import feast.options.OptionsParser; -import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; - -@Slf4j -public class PostgresOptionsTest { - @Test - public void testJdbcOptionsUrl() { - PostgresOptions options = - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .build(), - PostgresOptions.class); - assertEquals("jdbc:postgresql://host:port/database", options.url); - } - - @Test(expected = IllegalArgumentException.class) - public void testJdbcOptionsUrlInvalid() { - PostgresOptions options = - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:missingpostgresprefix://fadfas") - .build(), - PostgresOptions.class); - } - - @Test - public void testJdbcOptionsPrefix() { - assertEquals( - "abcd", - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "abcd") - .build(), - PostgresOptions.class) - .prefix); - - assertEquals( - "ABCD", - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "ABCD") - .build(), - PostgresOptions.class) - .prefix); - - assertEquals( - "_abc", - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "_abc") - .build(), - PostgresOptions.class) - .prefix); - - assertEquals( - "abcd_ABCD_1234", - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "abcd_ABCD_1234") - .build(), - PostgresOptions.class) - .prefix); - } - - /** - * Test that invalid prefix is rejected - */ - @Test - public void testJdbcOptionsPrefixInvalid() { - try { - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "9123abcd") - .build(), - PostgresOptions.class); - fail(); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("property \"prefix\" must match")); - } - - try { - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "-asdf") - .build(), - PostgresOptions.class); - fail(); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("property \"prefix\" must match")); - } - - try { - OptionsParser.parse( - ImmutableMap.builder() - .put("postgres.url", "jdbc:postgresql://host:port/database") - .put("postgres.prefix", "$asdf") - .build(), - PostgresOptions.class); - fail(); - } catch (IllegalArgumentException e) { - assertTrue(e.getMessage().contains("property \"prefix\" must match")); - } - } -} From b6e05b1db67c981cc00f7387474de5816ff91277 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Sun, 27 Jan 2019 22:56:26 +0800 Subject: [PATCH 07/15] do not write historical values to redis serving store --- .../redis/FeatureRowToRedisMutationDoFn.java | 25 --------- .../redis/FeatureRowRedisIOWriteTest.java | 51 ------------------- 2 files changed, 76 deletions(-) diff --git a/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java b/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java index 2ee46a93825..9c0537b3595 100644 --- a/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java +++ b/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java @@ -110,31 +110,6 @@ public void processElement(ProcessContext context) { .expiryMillis(expiryMillis) .method(Method.SET) .build()); - - if (!featureSpec.getGranularity().equals(Granularity.Enum.NONE)) { - long bucketId = getBucketId(roundedTimestamp, options.getBucketSizeDuration()); - - RedisBucketKey keyForValue = - getRedisBucketKey(entityKey, featureIdHash, roundedTimestamp.getSeconds()); - - context.output( - RedisMutation.builder() - .key(keyForValue.toByteArray()) - .value(value.toByteArray()) - .expiryMillis(expiryMillis) - .method(Method.SET) - .build()); - - RedisBucketKey keyForZset = getRedisBucketKey(entityKey, featureIdHash, bucketId); - context.output( - RedisMutation.builder() - .key(keyForZset.toByteArray()) - .value(keyForValue.toByteArray()) - .score(roundedTimestamp.getSeconds()) - .expiryMillis(expiryMillis) - .method(Method.ZADD) - .build()); - } } } } diff --git a/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java b/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java index eb75560adfb..d8aa81ab0f5 100644 --- a/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java +++ b/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java @@ -17,7 +17,6 @@ package feast.storage.redis; -import static feast.storage.redis.FeatureRowToRedisMutationDoFn.getBucketId; import static feast.storage.redis.FeatureRowToRedisMutationDoFn.getFeatureIdSha1Prefix; import static feast.storage.redis.FeatureRowToRedisMutationDoFn.getRedisBucketKey; import static org.junit.Assert.assertEquals; @@ -32,7 +31,6 @@ import feast.ingestion.service.SpecService; import feast.ingestion.transform.FeatureIO; import feast.ingestion.util.DateUtil; -import feast.options.OptionsParser; import feast.specs.ImportSpecProto.Field; import feast.specs.ImportSpecProto.ImportSpec; import feast.specs.ImportSpecProto.Schema; @@ -51,8 +49,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; import org.joda.time.DateTime; -import org.joda.time.Duration; -import org.joda.time.format.ISOPeriodFormat; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; @@ -228,55 +224,8 @@ public void testWriteHourGranularity() throws IOException { RedisBucketKey featureStringLatestKey = getRedisBucketKey("1", getFeatureIdSha1Prefix(featureHourString), 0L); - RedisBucketKey featureInt32ValueKey = - getRedisBucketKey("1", getFeatureIdSha1Prefix(featureHourInt32), - roundedTimestamp.getSeconds()); - RedisBucketKey featureStringValueKey = - getRedisBucketKey( - "1", getFeatureIdSha1Prefix(featureHourString), roundedTimestamp.getSeconds()); - - // TODO have a helper func for loading feature store options - Duration featureInt32BucketSize = - OptionsParser.parse( - specs.getFeatureSpec(featureHourInt32).getDataStores().getServing().getOptionsMap(), - RedisFeatureOptions.class) - .getBucketSizeDuration(); - - RedisBucketKey featureInt32BucketKey = - getRedisBucketKey( - "1", - getFeatureIdSha1Prefix(featureHourInt32), - getBucketId(roundedTimestamp, featureInt32BucketSize)); - RedisBucketKey featureStringBucketKey = - getRedisBucketKey( - "1", - getFeatureIdSha1Prefix(featureHourString), - // No bucketsize specified so uses the default. - getBucketId( - roundedTimestamp, - ISOPeriodFormat.standard() - .parsePeriod(RedisFeatureOptions.DEFAULT_BUCKET_SIZE) - .toStandardDuration())); - checkRedisValue(featureInt32LatestKey, Values.ofInt32(1), roundedTimestamp); checkRedisValue(featureStringLatestKey, Values.ofString("a"), roundedTimestamp); - - checkRedisValue(featureInt32ValueKey, Values.ofInt32(1), roundedTimestamp); - checkRedisValue(featureStringValueKey, Values.ofString("a"), roundedTimestamp); - - checkRedisValue(featureInt32BucketKey, featureInt32ValueKey, roundedTimestamp); - checkRedisValue(featureStringBucketKey, featureStringValueKey, roundedTimestamp); - } - - /** - * Check that a key's value is another key. - */ - void checkRedisValue(RedisBucketKey key, RedisBucketKey expectedValue, Timestamp timestamp) - throws InvalidProtocolBufferException { - Set result = jedis.zrangeByScore(key.toByteArray(), 0, Long.MAX_VALUE); - assertEquals(1, result.size()); - RedisBucketKey value = RedisBucketKey.parseFrom(result.iterator().next()); - assertEquals(expectedValue, value); } void checkRedisValue(RedisBucketKey key, Value expectedValue, Timestamp expectedTimestamp) From 301d7fcded9fb297771e32f87cbbe372f34acfea Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Sun, 27 Jan 2019 23:03:02 +0800 Subject: [PATCH 08/15] formatting and fix test --- .../redis/FeatureRowToRedisMutationDoFn.java | 13 +++++++------ .../feast/ingestion/ImportJobCSVTest.java | 19 ++----------------- .../redis/FeatureRowRedisIOWriteTest.java | 1 - .../FeatureRowToRedisMutationDoFnTest.java | 7 ++++--- .../storage/redis/RedisStoreOptionsTest.java | 2 +- 5 files changed, 14 insertions(+), 28 deletions(-) diff --git a/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java b/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java index 9c0537b3595..dad225d7ba8 100644 --- a/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java +++ b/ingestion/src/main/java/feast/storage/redis/FeatureRowToRedisMutationDoFn.java @@ -19,10 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import java.util.Random; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.commons.codec.digest.DigestUtils; -import org.joda.time.Duration; import feast.SerializableCache; import feast.ingestion.model.Specs; import feast.ingestion.util.DateUtil; @@ -35,7 +31,10 @@ import feast.types.FeatureProto.Feature; import feast.types.FeatureRowExtendedProto.FeatureRowExtended; import feast.types.FeatureRowProto.FeatureRow; -import feast.types.GranularityProto.Granularity; +import java.util.Random; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.commons.codec.digest.DigestUtils; +import org.joda.time.Duration; public class FeatureRowToRedisMutationDoFn extends DoFn { @@ -75,7 +74,9 @@ static String getFeatureIdSha1Prefix(String featureId) { return DigestUtils.sha1Hex(featureId.getBytes()).substring(0, 7); } - /** Output a redis mutation object for every feature in the feature row. */ + /** + * Output a redis mutation object for every feature in the feature row. + */ @ProcessElement public void processElement(ProcessContext context) { FeatureRowExtended rowExtended = context.element(); diff --git a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java index 8dc238b8a3a..bd7460e6372 100644 --- a/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java +++ b/ingestion/src/test/java/feast/ingestion/ImportJobCSVTest.java @@ -224,25 +224,10 @@ public void testImportCSV_withSample1() throws IOException { PCollectionList.of(ErrorsStoreService.get(MockErrorsStore.class).getWrite().getInputs()) .apply("flatten errors input", Flatten.pCollections()); - List expectedRows = - Lists.newArrayList( - normalize( - FeatureRow.newBuilder() - .setGranularity(Granularity.Enum.NONE) - .setEventTimestamp(Timestamp.getDefaultInstance()) - .setEntityKey("1") - .setEntityName("testEntity") - .addFeatures(Features.of("testEntity.none.testInt32", Values.ofInt32(101))) - .addFeatures(Features.of("testEntity.none.testString", Values.ofString("a"))) - .build())); - + PAssert.that(writtenToServing).satisfies(hasCount(1)); + PAssert.that(writtenToWarehouse).satisfies(hasCount(1)); PAssert.that(writtenToErrors).satisfies(hasCount(0)); - PAssert.that(writtenToServing.apply("serving toFeatureRows", new ToOrderedFeatureRows())) - .containsInAnyOrder(expectedRows); - - PAssert.that(writtenToWarehouse.apply("warehouse toFeatureRows", new ToOrderedFeatureRows())) - .containsInAnyOrder(expectedRows); testPipeline.run(); } diff --git a/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java b/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java index d8aa81ab0f5..591736c1bed 100644 --- a/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java +++ b/ingestion/src/test/java/feast/storage/redis/FeatureRowRedisIOWriteTest.java @@ -44,7 +44,6 @@ import java.io.IOException; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.Set; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; diff --git a/ingestion/src/test/java/feast/storage/redis/FeatureRowToRedisMutationDoFnTest.java b/ingestion/src/test/java/feast/storage/redis/FeatureRowToRedisMutationDoFnTest.java index f2ca4487dc5..242042fb357 100644 --- a/ingestion/src/test/java/feast/storage/redis/FeatureRowToRedisMutationDoFnTest.java +++ b/ingestion/src/test/java/feast/storage/redis/FeatureRowToRedisMutationDoFnTest.java @@ -17,6 +17,8 @@ package feast.storage.redis; +import feast.ingestion.util.DateUtil; +import feast.storage.RedisProto.RedisBucketKey; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.commons.codec.digest.DigestUtils; @@ -25,12 +27,11 @@ import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import feast.ingestion.util.DateUtil; -import feast.storage.RedisProto.RedisBucketKey; public class FeatureRowToRedisMutationDoFnTest { - @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); @Test public void testRedisBucketKeySize() { diff --git a/ingestion/src/test/java/feast/storage/redis/RedisStoreOptionsTest.java b/ingestion/src/test/java/feast/storage/redis/RedisStoreOptionsTest.java index cc94bfa98af..87b7fc98fad 100644 --- a/ingestion/src/test/java/feast/storage/redis/RedisStoreOptionsTest.java +++ b/ingestion/src/test/java/feast/storage/redis/RedisStoreOptionsTest.java @@ -19,10 +19,10 @@ import static org.junit.Assert.assertEquals; +import feast.options.OptionsParser; import java.util.HashMap; import java.util.Map; import org.junit.Test; -import feast.options.OptionsParser; public class RedisStoreOptionsTest { From d8ef7491c6d0c5db65a17ca23e926ba688b54966 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Sun, 27 Jan 2019 23:15:34 +0800 Subject: [PATCH 09/15] core importspec validator test --- .../feast/core/validators/SpecValidator.java | 22 +++++++- .../core/validators/SpecValidatorTest.java | 55 +++++++++++++++++++ 2 files changed, 75 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/feast/core/validators/SpecValidator.java b/core/src/main/java/feast/core/validators/SpecValidator.java index 8259607e5e4..18095b25914 100644 --- a/core/src/main/java/feast/core/validators/SpecValidator.java +++ b/core/src/main/java/feast/core/validators/SpecValidator.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static feast.core.validators.Matchers.checkLowerSnakeCase; +import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Lists; import feast.core.dao.EntityInfoRepository; @@ -40,6 +41,8 @@ import feast.specs.ImportSpecProto.ImportSpec; import feast.specs.StorageSpecProto.StorageSpec; import java.util.Arrays; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -256,6 +259,19 @@ public void validateImportSpec(ImportSpec spec) throws IllegalArgumentException entityInfoRepository.existsById(name), Strings.lenientFormat("Entity %s not registered", name)); } + Map jobOptions = spec.getJobOptionsMap(); + if (jobOptions.size() > 0) { + List opts = Lists.newArrayList( + "sample.limit", + "coalesceRows.enabled", + "coalesceRows.delaySeconds", + "coalesceRows.timeoutSeconds" + ); + for (String key : jobOptions.keySet()) { + Preconditions.checkArgument(opts.contains(key), + Strings.lenientFormat("Option %s is not a valid jobOption", key)); + } + } } catch (NullPointerException | IllegalArgumentException e) { throw new IllegalArgumentException( Strings.lenientFormat("Validation for import spec failed: %s", e.getMessage())); @@ -278,7 +294,8 @@ private void checkKafkaImportSpecOption(ImportSpec spec) { private void checkFileImportSpecOption(ImportSpec spec) throws IllegalArgumentException { try { - checkArgument(!spec.getSourceOptionsOrDefault("path", "").equals(""), "File path cannot be empty"); + checkArgument(!spec.getSourceOptionsOrDefault("path", "").equals(""), + "File path cannot be empty"); } catch (NullPointerException | IllegalArgumentException e) { throw new IllegalArgumentException( Strings.lenientFormat("Invalid options: %s", e.getMessage())); @@ -305,7 +322,8 @@ private void checkBigqueryImportSpecOption(ImportSpec spec) throws IllegalArgume "Bigquery project cannot be empty"); checkArgument(!spec.getSourceOptionsOrThrow("dataset").equals(""), "Bigquery dataset cannot be empty"); - checkArgument(!spec.getSourceOptionsOrThrow("table").equals(""), "Bigquery table cannot be empty"); + checkArgument(!spec.getSourceOptionsOrThrow("table").equals(""), + "Bigquery table cannot be empty"); } catch (NullPointerException | IllegalArgumentException e) { throw new IllegalArgumentException( Strings.lenientFormat("Invalid options: %s", e.getMessage())); diff --git a/core/src/test/java/feast/core/validators/SpecValidatorTest.java b/core/src/test/java/feast/core/validators/SpecValidatorTest.java index 3cf2caf6782..052fa609508 100644 --- a/core/src/test/java/feast/core/validators/SpecValidatorTest.java +++ b/core/src/test/java/feast/core/validators/SpecValidatorTest.java @@ -810,6 +810,61 @@ public void importSpecWithKafkaSourceAndCorrectOptionsShouldPassValidation() { validator.validateImportSpec(input); } + @Test + public void importSpecWithCoalesceJobOptionsShouldPassValidation() { + SpecValidator validator = + new SpecValidator( + storageInfoRepository, + entityInfoRepository, + featureGroupInfoRepository, + featureInfoRepository); + when(featureInfoRepository.existsById("some_existing_feature")).thenReturn(true); + when(entityInfoRepository.existsById("someEntity")).thenReturn(true); + Schema schema = + Schema.newBuilder() + .addFields(Field.newBuilder().setFeatureId("some_existing_feature").build()) + .build(); + ImportSpec input = + ImportSpec.newBuilder() + .setType("kafka") + .putSourceOptions("topics", "my-kafka-topic") + .putSourceOptions("server", "localhost:54321") + .putJobOptions("coalesceRows.enabled", "true") + .putJobOptions("coalesceRows.delaySeconds", "10000") + .putJobOptions("coalesceRows.timeoutSeconds", "20000") + .putJobOptions("sample.limit", "1000") + .setSchema(schema) + .addEntities("someEntity") + .build(); + validator.validateImportSpec(input); + } + + @Test + public void importSpecWithLimitJobOptionsShouldPassValidation() { + SpecValidator validator = + new SpecValidator( + storageInfoRepository, + entityInfoRepository, + featureGroupInfoRepository, + featureInfoRepository); + when(featureInfoRepository.existsById("some_existing_feature")).thenReturn(true); + when(entityInfoRepository.existsById("someEntity")).thenReturn(true); + Schema schema = + Schema.newBuilder() + .addFields(Field.newBuilder().setFeatureId("some_existing_feature").build()) + .build(); + ImportSpec input = + ImportSpec.newBuilder() + .setType("kafka") + .putSourceOptions("topics", "my-kafka-topic") + .putSourceOptions("server", "localhost:54321") + .putJobOptions("sample.limit", "1000") + .setSchema(schema) + .addEntities("someEntity") + .build(); + validator.validateImportSpec(input); + } + @Test public void importSpecWithKafkaSourceWithoutOptionsShouldThrowIllegalArgumentException() { SpecValidator validator = From 4371ce23f4ddae53df7b73a8a3fc34aca5fc9d8d Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Sun, 27 Jan 2019 23:17:00 +0800 Subject: [PATCH 10/15] fix typo on option name --- ingestion/src/main/java/feast/ingestion/options/JobOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java index 68017644e82..833b03bdb80 100644 --- a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java +++ b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java @@ -38,7 +38,7 @@ public long getSampleLimit() { return sampleLimit; } - public void setSampleimit(long sampleLimit) { + public void setSampleLimit(long sampleLimit) { this.sampleLimit = sampleLimit; } From 8fe8b5c5baf04221e27ef9aad09e6070746ab595 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Sun, 27 Jan 2019 23:19:33 +0800 Subject: [PATCH 11/15] use Timestamps.compare --- .../feast/ingestion/transform/CoalesceFeatureRows.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java index 3b02834d778..43d418fa539 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java @@ -18,12 +18,11 @@ package feast.ingestion.transform; import com.google.common.base.Preconditions; -import com.google.protobuf.Timestamp; +import com.google.protobuf.util.Timestamps; import feast.types.FeatureProto.Feature; import feast.types.FeatureRowProto.FeatureRow; import feast_ingestion.types.CoalesceAccumProto.CoalesceAccum; import feast_ingestion.types.CoalesceKeyProto.CoalesceKey; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -66,9 +65,6 @@ public class CoalesceFeatureRows extends PTransform, PCollection> { - private static final Comparator TIMESTAMP_COMPARATOR = Comparator - .comparing(Timestamp::getSeconds) - .thenComparing(Timestamp::getNanos); private static final SerializableFunction KEY_FUNCTION = (row) -> CoalesceKey.newBuilder() .setEntityName(row.getEntityName()) @@ -134,7 +130,7 @@ public static CoalesceAccum combineFeatureRows(CoalesceAccum seed, Iterable Date: Sun, 27 Jan 2019 23:21:54 +0800 Subject: [PATCH 12/15] remove redudant proto message --- protos/feast/specs/ImportSpec.proto | 5 ----- 1 file changed, 5 deletions(-) diff --git a/protos/feast/specs/ImportSpec.proto b/protos/feast/specs/ImportSpec.proto index a1b2d6273b8..a6cab4f71d7 100644 --- a/protos/feast/specs/ImportSpec.proto +++ b/protos/feast/specs/ImportSpec.proto @@ -36,11 +36,6 @@ message ImportSpec { Schema schema = 4; } -message ImportJobOptions { - bool coalesceRows = 1; - int64 coalesceRowsDelaySeconds = 2; -} - message Schema { repeated Field fields = 1; // the event timestamp to set per row. From ca9332ec763f0baecbee148dfbad46f9288e70b5 Mon Sep 17 00:00:00 2001 From: Tim Sell Date: Fri, 8 Feb 2019 15:36:32 +0800 Subject: [PATCH 13/15] default coalescerows to true --- .../src/main/java/feast/ingestion/options/JobOptions.java | 2 +- .../java/feast/ingestion/transform/CoalesceFeatureRows.java | 1 - .../test/java/feast/ingestion/options/JobOptionsTest.java | 6 +++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java index 833b03bdb80..c316f749bce 100644 --- a/ingestion/src/main/java/feast/ingestion/options/JobOptions.java +++ b/ingestion/src/main/java/feast/ingestion/options/JobOptions.java @@ -28,7 +28,7 @@ public class JobOptions implements Options { private long sampleLimit; - private boolean coalesceRowsEnabled; + private boolean coalesceRowsEnabled = true; private long coalesceRowsDelaySeconds; private long coalesceRowsTimeoutSeconds; diff --git a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java index 43d418fa539..a7c53d21c65 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java +++ b/ingestion/src/main/java/feast/ingestion/transform/CoalesceFeatureRows.java @@ -78,7 +78,6 @@ public class CoalesceFeatureRows extends CoalesceFeatureRows() { this(0, 0); - } public CoalesceFeatureRows(long delaySeconds, long timeoutSeconds) { diff --git a/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java b/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java index aacc83886a2..59f4a478103 100644 --- a/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java +++ b/ingestion/src/test/java/feast/ingestion/options/JobOptionsTest.java @@ -31,7 +31,7 @@ public class JobOptionsTest { @Test public void test_shouldParseAll() { Map map = new HashMap<>(); - map.put("coalesceRows.enabled", "true"); + map.put("coalesceRows.enabled", "false"); map.put("coalesceRows.delaySeconds", "123"); map.put("coalesceRows.timeoutSeconds", "1800"); map.put("sample.limit", "1234"); @@ -40,7 +40,7 @@ public void test_shouldParseAll() { JobOptions options = OptionsParser.parse(map, JobOptions.class); assertEquals(options.getSampleLimit(), 1234L); assertEquals(options.getCoalesceRowsDelaySeconds(), 123L); - assertTrue(options.isCoalesceRowsEnabled()); + assertFalse(options.isCoalesceRowsEnabled()); assertEquals(options.getCoalesceRowsTimeoutSeconds(), 1800L); } @@ -49,7 +49,7 @@ public void test_shouldParseEmptyOptions() { JobOptions options = OptionsParser.parse(new HashMap<>(), JobOptions.class); assertEquals(options.getSampleLimit(), 0L); assertEquals(options.getCoalesceRowsDelaySeconds(), 0L); - assertFalse(options.isCoalesceRowsEnabled()); + assertTrue(options.isCoalesceRowsEnabled()); //defaults to true assertEquals(options.getCoalesceRowsTimeoutSeconds(), 0L); } From 4f701141129a24f904a2de1b76cca561094d4b72 Mon Sep 17 00:00:00 2001 From: zhilingc Date: Mon, 11 Feb 2019 01:03:06 +0800 Subject: [PATCH 14/15] Update go protos and cli tests --- cli/feast/pkg/parse/yaml.go | 6 - cli/feast/pkg/parse/yaml_test.go | 55 +-- .../go/feast/core/DatasetService.pb.go | 388 ++++++++++++++++++ .../generated/go/feast/specs/ImportSpec.pb.go | 95 +++-- .../generated/go/feast/storage/BigTable.pb.go | 34 +- protos/generated/go/feast/types/Feature.pb.go | 33 +- .../generated/go/feast/types/FeatureRow.pb.go | 54 +-- 7 files changed, 504 insertions(+), 161 deletions(-) create mode 100644 protos/generated/go/feast/core/DatasetService.pb.go diff --git a/cli/feast/pkg/parse/yaml.go b/cli/feast/pkg/parse/yaml.go index 503f0acb494..f02698171e1 100644 --- a/cli/feast/pkg/parse/yaml.go +++ b/cli/feast/pkg/parse/yaml.go @@ -16,7 +16,6 @@ package parse import ( "encoding/json" - "fmt" "time" "github.com/golang/protobuf/ptypes/timestamp" @@ -98,11 +97,6 @@ func YamlToImportSpec(in []byte) (*specs.ImportSpec, error) { return nil, err } - // schema must be available for 'file' or 'bigquery' - if (ymlMap["type"] == "file" || ymlMap["type"] == "bigquery") && ymlMap["schema"] == nil { - return nil, fmt.Errorf("Schema must be specified for importing data from file or BigQuery") - } - // either timestampValue or timestampColumn var timestampValue *timestamp.Timestamp var timestampColumn string diff --git a/cli/feast/pkg/parse/yaml_test.go b/cli/feast/pkg/parse/yaml_test.go index fbb43ef8ee9..00331668ea3 100644 --- a/cli/feast/pkg/parse/yaml_test.go +++ b/cli/feast/pkg/parse/yaml_test.go @@ -243,7 +243,9 @@ func TestYamlToImportSpec(t *testing.T) { { name: "valid yaml", input: []byte(`type: file -options: +jobOptions: + coalesceRows.enabled: "true" +sourceOptions: format: csv path: jaeger_last_opportunity_sample.csv entities: @@ -258,7 +260,10 @@ schema: featureId: driver.none.last_opportunity`), expected: &specs.ImportSpec{ Type: "file", - Options: map[string]string{ + JobOptions: map[string]string{ + "coalesceRows.enabled": "true", + }, + SourceOptions: map[string]string{ "format": "csv", "path": "jaeger_last_opportunity_sample.csv", }, @@ -299,49 +304,3 @@ schema: }) } } - -func TestYamlToImportSpecNoSchema(t *testing.T) { - tt := []struct { - name string - input []byte - expected *specs.ImportSpec - err error - }{ - { - name: "valid yaml", - input: []byte(`type: pubsub -options: - topic: projects/your-gcp-project/topics/feast-test -entities: - - testentity`), - expected: &specs.ImportSpec{ - Type: "pubsub", - Options: map[string]string{ - "topic": "projects/your-gcp-project/topics/feast-test", - }, - Entities: []string{"testentity"}, - }, - err: nil, - }, - } - - for _, tc := range tt { - t.Run(tc.name, func(t *testing.T) { - spec, err := YamlToImportSpec(tc.input) - if tc.err == nil { - if err != nil { - t.Error(err) - } else if !cmp.Equal(spec, tc.expected) { - t.Errorf("Expected %s, got %s", tc.expected, spec) - } - } else { - // we expect an error - if err == nil { - t.Error(err) - } else if err.Error() != tc.err.Error() { - t.Errorf("Expected error %v, got %v", err.Error(), tc.err.Error()) - } - } - }) - } -} diff --git a/protos/generated/go/feast/core/DatasetService.pb.go b/protos/generated/go/feast/core/DatasetService.pb.go new file mode 100644 index 00000000000..d2d3b01c932 --- /dev/null +++ b/protos/generated/go/feast/core/DatasetService.pb.go @@ -0,0 +1,388 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// source: feast/core/DatasetService.proto + +package core // import "github.com/gojek/feast/protos/generated/go/feast/core" + +import proto "github.com/golang/protobuf/proto" +import fmt "fmt" +import math "math" +import timestamp "github.com/golang/protobuf/ptypes/timestamp" + +import ( + context "golang.org/x/net/context" + grpc "google.golang.org/grpc" +) + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package + +type DatasetServiceTypes struct { + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DatasetServiceTypes) Reset() { *m = DatasetServiceTypes{} } +func (m *DatasetServiceTypes) String() string { return proto.CompactTextString(m) } +func (*DatasetServiceTypes) ProtoMessage() {} +func (*DatasetServiceTypes) Descriptor() ([]byte, []int) { + return fileDescriptor_DatasetService_37ae639a8c7b5dd5, []int{0} +} +func (m *DatasetServiceTypes) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DatasetServiceTypes.Unmarshal(m, b) +} +func (m *DatasetServiceTypes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DatasetServiceTypes.Marshal(b, m, deterministic) +} +func (dst *DatasetServiceTypes) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatasetServiceTypes.Merge(dst, src) +} +func (m *DatasetServiceTypes) XXX_Size() int { + return xxx_messageInfo_DatasetServiceTypes.Size(m) +} +func (m *DatasetServiceTypes) XXX_DiscardUnknown() { + xxx_messageInfo_DatasetServiceTypes.DiscardUnknown(m) +} + +var xxx_messageInfo_DatasetServiceTypes proto.InternalMessageInfo + +type DatasetServiceTypes_CreateDatasetRequest struct { + // set of features for which its training data should be created + FeatureSet *FeatureSet `protobuf:"bytes,1,opt,name=featureSet,proto3" json:"featureSet,omitempty"` + // start date of the training data (inclusive) + StartDate *timestamp.Timestamp `protobuf:"bytes,2,opt,name=startDate,proto3" json:"startDate,omitempty"` + // end date of the training data (inclusive) + EndDate *timestamp.Timestamp `protobuf:"bytes,3,opt,name=endDate,proto3" json:"endDate,omitempty"` + // (optional) number of row that should be generated + // (default: none) + Limit int64 `protobuf:"varint,4,opt,name=limit,proto3" json:"limit,omitempty"` + // (optional) prefix for dataset name + NamePrefix string `protobuf:"bytes,5,opt,name=namePrefix,proto3" json:"namePrefix,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DatasetServiceTypes_CreateDatasetRequest) Reset() { + *m = DatasetServiceTypes_CreateDatasetRequest{} +} +func (m *DatasetServiceTypes_CreateDatasetRequest) String() string { return proto.CompactTextString(m) } +func (*DatasetServiceTypes_CreateDatasetRequest) ProtoMessage() {} +func (*DatasetServiceTypes_CreateDatasetRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_DatasetService_37ae639a8c7b5dd5, []int{0, 0} +} +func (m *DatasetServiceTypes_CreateDatasetRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DatasetServiceTypes_CreateDatasetRequest.Unmarshal(m, b) +} +func (m *DatasetServiceTypes_CreateDatasetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DatasetServiceTypes_CreateDatasetRequest.Marshal(b, m, deterministic) +} +func (dst *DatasetServiceTypes_CreateDatasetRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatasetServiceTypes_CreateDatasetRequest.Merge(dst, src) +} +func (m *DatasetServiceTypes_CreateDatasetRequest) XXX_Size() int { + return xxx_messageInfo_DatasetServiceTypes_CreateDatasetRequest.Size(m) +} +func (m *DatasetServiceTypes_CreateDatasetRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DatasetServiceTypes_CreateDatasetRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DatasetServiceTypes_CreateDatasetRequest proto.InternalMessageInfo + +func (m *DatasetServiceTypes_CreateDatasetRequest) GetFeatureSet() *FeatureSet { + if m != nil { + return m.FeatureSet + } + return nil +} + +func (m *DatasetServiceTypes_CreateDatasetRequest) GetStartDate() *timestamp.Timestamp { + if m != nil { + return m.StartDate + } + return nil +} + +func (m *DatasetServiceTypes_CreateDatasetRequest) GetEndDate() *timestamp.Timestamp { + if m != nil { + return m.EndDate + } + return nil +} + +func (m *DatasetServiceTypes_CreateDatasetRequest) GetLimit() int64 { + if m != nil { + return m.Limit + } + return 0 +} + +func (m *DatasetServiceTypes_CreateDatasetRequest) GetNamePrefix() string { + if m != nil { + return m.NamePrefix + } + return "" +} + +type DatasetServiceTypes_CreateDatasetResponse struct { + // information of the created training dataset + DatasetInfo *DatasetInfo `protobuf:"bytes,1,opt,name=datasetInfo,proto3" json:"datasetInfo,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DatasetServiceTypes_CreateDatasetResponse) Reset() { + *m = DatasetServiceTypes_CreateDatasetResponse{} +} +func (m *DatasetServiceTypes_CreateDatasetResponse) String() string { return proto.CompactTextString(m) } +func (*DatasetServiceTypes_CreateDatasetResponse) ProtoMessage() {} +func (*DatasetServiceTypes_CreateDatasetResponse) Descriptor() ([]byte, []int) { + return fileDescriptor_DatasetService_37ae639a8c7b5dd5, []int{0, 1} +} +func (m *DatasetServiceTypes_CreateDatasetResponse) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DatasetServiceTypes_CreateDatasetResponse.Unmarshal(m, b) +} +func (m *DatasetServiceTypes_CreateDatasetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DatasetServiceTypes_CreateDatasetResponse.Marshal(b, m, deterministic) +} +func (dst *DatasetServiceTypes_CreateDatasetResponse) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatasetServiceTypes_CreateDatasetResponse.Merge(dst, src) +} +func (m *DatasetServiceTypes_CreateDatasetResponse) XXX_Size() int { + return xxx_messageInfo_DatasetServiceTypes_CreateDatasetResponse.Size(m) +} +func (m *DatasetServiceTypes_CreateDatasetResponse) XXX_DiscardUnknown() { + xxx_messageInfo_DatasetServiceTypes_CreateDatasetResponse.DiscardUnknown(m) +} + +var xxx_messageInfo_DatasetServiceTypes_CreateDatasetResponse proto.InternalMessageInfo + +func (m *DatasetServiceTypes_CreateDatasetResponse) GetDatasetInfo() *DatasetInfo { + if m != nil { + return m.DatasetInfo + } + return nil +} + +// Represent a collection of feature having same entity name +type FeatureSet struct { + // entity related to this feature set + EntityName string `protobuf:"bytes,1,opt,name=entityName,proto3" json:"entityName,omitempty"` + // list of feature id in this feature set + FeatureIds []string `protobuf:"bytes,2,rep,name=featureIds,proto3" json:"featureIds,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *FeatureSet) Reset() { *m = FeatureSet{} } +func (m *FeatureSet) String() string { return proto.CompactTextString(m) } +func (*FeatureSet) ProtoMessage() {} +func (*FeatureSet) Descriptor() ([]byte, []int) { + return fileDescriptor_DatasetService_37ae639a8c7b5dd5, []int{1} +} +func (m *FeatureSet) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_FeatureSet.Unmarshal(m, b) +} +func (m *FeatureSet) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_FeatureSet.Marshal(b, m, deterministic) +} +func (dst *FeatureSet) XXX_Merge(src proto.Message) { + xxx_messageInfo_FeatureSet.Merge(dst, src) +} +func (m *FeatureSet) XXX_Size() int { + return xxx_messageInfo_FeatureSet.Size(m) +} +func (m *FeatureSet) XXX_DiscardUnknown() { + xxx_messageInfo_FeatureSet.DiscardUnknown(m) +} + +var xxx_messageInfo_FeatureSet proto.InternalMessageInfo + +func (m *FeatureSet) GetEntityName() string { + if m != nil { + return m.EntityName + } + return "" +} + +func (m *FeatureSet) GetFeatureIds() []string { + if m != nil { + return m.FeatureIds + } + return nil +} + +// Representation of training dataset information +type DatasetInfo struct { + // name of dataset + Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"` + // URL to table location of the training dataset + TableUrl string `protobuf:"bytes,2,opt,name=tableUrl,proto3" json:"tableUrl,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DatasetInfo) Reset() { *m = DatasetInfo{} } +func (m *DatasetInfo) String() string { return proto.CompactTextString(m) } +func (*DatasetInfo) ProtoMessage() {} +func (*DatasetInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_DatasetService_37ae639a8c7b5dd5, []int{2} +} +func (m *DatasetInfo) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DatasetInfo.Unmarshal(m, b) +} +func (m *DatasetInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DatasetInfo.Marshal(b, m, deterministic) +} +func (dst *DatasetInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_DatasetInfo.Merge(dst, src) +} +func (m *DatasetInfo) XXX_Size() int { + return xxx_messageInfo_DatasetInfo.Size(m) +} +func (m *DatasetInfo) XXX_DiscardUnknown() { + xxx_messageInfo_DatasetInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_DatasetInfo proto.InternalMessageInfo + +func (m *DatasetInfo) GetName() string { + if m != nil { + return m.Name + } + return "" +} + +func (m *DatasetInfo) GetTableUrl() string { + if m != nil { + return m.TableUrl + } + return "" +} + +func init() { + proto.RegisterType((*DatasetServiceTypes)(nil), "feast.core.DatasetServiceTypes") + proto.RegisterType((*DatasetServiceTypes_CreateDatasetRequest)(nil), "feast.core.DatasetServiceTypes.CreateDatasetRequest") + proto.RegisterType((*DatasetServiceTypes_CreateDatasetResponse)(nil), "feast.core.DatasetServiceTypes.CreateDatasetResponse") + proto.RegisterType((*FeatureSet)(nil), "feast.core.FeatureSet") + proto.RegisterType((*DatasetInfo)(nil), "feast.core.DatasetInfo") +} + +// Reference imports to suppress errors if they are not otherwise used. +var _ context.Context +var _ grpc.ClientConn + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +const _ = grpc.SupportPackageIsVersion4 + +// DatasetServiceClient is the client API for DatasetService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. +type DatasetServiceClient interface { + // Create training dataset for a feature set + CreateDataset(ctx context.Context, in *DatasetServiceTypes_CreateDatasetRequest, opts ...grpc.CallOption) (*DatasetServiceTypes_CreateDatasetResponse, error) +} + +type datasetServiceClient struct { + cc *grpc.ClientConn +} + +func NewDatasetServiceClient(cc *grpc.ClientConn) DatasetServiceClient { + return &datasetServiceClient{cc} +} + +func (c *datasetServiceClient) CreateDataset(ctx context.Context, in *DatasetServiceTypes_CreateDatasetRequest, opts ...grpc.CallOption) (*DatasetServiceTypes_CreateDatasetResponse, error) { + out := new(DatasetServiceTypes_CreateDatasetResponse) + err := c.cc.Invoke(ctx, "/feast.core.DatasetService/CreateDataset", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +// DatasetServiceServer is the server API for DatasetService service. +type DatasetServiceServer interface { + // Create training dataset for a feature set + CreateDataset(context.Context, *DatasetServiceTypes_CreateDatasetRequest) (*DatasetServiceTypes_CreateDatasetResponse, error) +} + +func RegisterDatasetServiceServer(s *grpc.Server, srv DatasetServiceServer) { + s.RegisterService(&_DatasetService_serviceDesc, srv) +} + +func _DatasetService_CreateDataset_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DatasetServiceTypes_CreateDatasetRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(DatasetServiceServer).CreateDataset(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/feast.core.DatasetService/CreateDataset", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(DatasetServiceServer).CreateDataset(ctx, req.(*DatasetServiceTypes_CreateDatasetRequest)) + } + return interceptor(ctx, in, info, handler) +} + +var _DatasetService_serviceDesc = grpc.ServiceDesc{ + ServiceName: "feast.core.DatasetService", + HandlerType: (*DatasetServiceServer)(nil), + Methods: []grpc.MethodDesc{ + { + MethodName: "CreateDataset", + Handler: _DatasetService_CreateDataset_Handler, + }, + }, + Streams: []grpc.StreamDesc{}, + Metadata: "feast/core/DatasetService.proto", +} + +func init() { + proto.RegisterFile("feast/core/DatasetService.proto", fileDescriptor_DatasetService_37ae639a8c7b5dd5) +} + +var fileDescriptor_DatasetService_37ae639a8c7b5dd5 = []byte{ + // 414 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x53, 0xc1, 0x6e, 0xd4, 0x30, + 0x10, 0x25, 0xbb, 0x2d, 0x90, 0x59, 0xc1, 0xc1, 0x14, 0x88, 0x72, 0xa0, 0x51, 0x4e, 0x7b, 0xb2, + 0xa5, 0xd2, 0x22, 0x38, 0x70, 0x29, 0x2b, 0xa4, 0x4a, 0x08, 0x55, 0x6e, 0x91, 0x10, 0x37, 0x67, + 0x33, 0x09, 0x81, 0x24, 0x0e, 0xf6, 0x04, 0xd1, 0x0b, 0xdf, 0xc0, 0x27, 0xf2, 0x11, 0x7c, 0x00, + 0x8a, 0xb3, 0xdb, 0x78, 0xab, 0x4a, 0x88, 0x9b, 0x3d, 0xef, 0xcd, 0x78, 0xde, 0xf8, 0x0d, 0x1c, + 0x16, 0xa8, 0x2c, 0x89, 0xb5, 0x36, 0x28, 0x56, 0x8a, 0x94, 0x45, 0xba, 0x40, 0xf3, 0xbd, 0x5a, + 0x23, 0xef, 0x8c, 0x26, 0xcd, 0xc0, 0x11, 0xf8, 0x40, 0x88, 0x0f, 0x4b, 0xad, 0xcb, 0x1a, 0x85, + 0x43, 0xb2, 0xbe, 0x10, 0x54, 0x35, 0x68, 0x49, 0x35, 0xdd, 0x48, 0x4e, 0x7f, 0xcf, 0xe0, 0xd1, + 0x6e, 0x95, 0xcb, 0xab, 0x0e, 0x6d, 0xfc, 0x27, 0x80, 0x83, 0x37, 0x06, 0x15, 0xe1, 0x06, 0x95, + 0xf8, 0xad, 0x47, 0x4b, 0xec, 0x05, 0x0c, 0xf5, 0xa9, 0x37, 0x78, 0x81, 0x14, 0x05, 0x49, 0xb0, + 0x5c, 0x1c, 0x3d, 0xe1, 0xd3, 0x93, 0xfc, 0xed, 0x35, 0x2a, 0x3d, 0x26, 0x7b, 0x09, 0xa1, 0x25, + 0x65, 0x68, 0xa5, 0x08, 0xa3, 0x99, 0x4b, 0x8b, 0xf9, 0xd8, 0x1d, 0xdf, 0x76, 0xc7, 0x2f, 0xb7, + 0xdd, 0xc9, 0x89, 0xcc, 0x8e, 0xe1, 0x1e, 0xb6, 0xb9, 0xcb, 0x9b, 0xff, 0x33, 0x6f, 0x4b, 0x65, + 0x07, 0xb0, 0x5f, 0x57, 0x4d, 0x45, 0xd1, 0x5e, 0x12, 0x2c, 0xe7, 0x72, 0xbc, 0xb0, 0x67, 0x00, + 0xad, 0x6a, 0xf0, 0xdc, 0x60, 0x51, 0xfd, 0x88, 0xf6, 0x93, 0x60, 0x19, 0x4a, 0x2f, 0x12, 0x4b, + 0x78, 0x7c, 0x43, 0xb5, 0xed, 0x74, 0x6b, 0x91, 0xbd, 0x82, 0x45, 0x3e, 0x86, 0xce, 0xda, 0x42, + 0x6f, 0x74, 0x3f, 0xf5, 0x75, 0xaf, 0x26, 0x58, 0xfa, 0xdc, 0xf4, 0x1d, 0xc0, 0x34, 0x93, 0xa1, + 0x03, 0x6c, 0xa9, 0xa2, 0xab, 0xf7, 0xaa, 0x41, 0x57, 0x27, 0x94, 0x5e, 0x64, 0xc0, 0x37, 0x53, + 0x3b, 0xcb, 0x6d, 0x34, 0x4b, 0xe6, 0x03, 0x3e, 0x45, 0xd2, 0xd7, 0xb0, 0xf0, 0x5e, 0x62, 0x0c, + 0xf6, 0xda, 0xa9, 0x90, 0x3b, 0xb3, 0x18, 0xee, 0x93, 0xca, 0x6a, 0xfc, 0x60, 0x6a, 0x37, 0xe9, + 0x50, 0x5e, 0xdf, 0x8f, 0x7e, 0x05, 0xf0, 0x70, 0xf7, 0xbf, 0xd9, 0x4f, 0x78, 0xb0, 0xa3, 0x99, + 0x1d, 0xdf, 0x22, 0xcb, 0x37, 0x07, 0xbf, 0xcd, 0x18, 0xf1, 0xc9, 0x7f, 0x66, 0x8d, 0x83, 0x4d, + 0xef, 0x9c, 0x7e, 0x04, 0xcf, 0xb1, 0xa7, 0x37, 0xdc, 0x78, 0x3e, 0x7c, 0xf1, 0xa7, 0x93, 0xb2, + 0xa2, 0xcf, 0x7d, 0xc6, 0xd7, 0xba, 0x11, 0xa5, 0xfe, 0x82, 0x5f, 0xc5, 0xb8, 0x04, 0xce, 0x00, + 0x56, 0x94, 0xd8, 0xa2, 0x51, 0x84, 0xb9, 0x28, 0xb5, 0x98, 0xd6, 0x23, 0xbb, 0xeb, 0xf0, 0xe7, + 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xbb, 0x9e, 0x4d, 0xe0, 0x33, 0x03, 0x00, 0x00, +} diff --git a/protos/generated/go/feast/specs/ImportSpec.pb.go b/protos/generated/go/feast/specs/ImportSpec.pb.go index d4cca689ba3..d16f9fa07bd 100644 --- a/protos/generated/go/feast/specs/ImportSpec.pb.go +++ b/protos/generated/go/feast/specs/ImportSpec.pb.go @@ -6,7 +6,6 @@ package specs // import "github.com/gojek/feast/protos/generated/go/feast/specs" import proto "github.com/golang/protobuf/proto" import fmt "fmt" import math "math" -import _ "github.com/gojek/feast/protos/generated/go/feast/types" import timestamp "github.com/golang/protobuf/ptypes/timestamp" // Reference imports to suppress errors if they are not otherwise used. @@ -21,13 +20,9 @@ var _ = math.Inf const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package type ImportSpec struct { - // type should define what sort of store it is - // pubsub: topic, subscription (already a featureRow) - // kafka: ? (already a featureRow) - // file: csv, parquet, json - // bigquery: table, query Type string `protobuf:"bytes,1,opt,name=type,proto3" json:"type,omitempty"` - Options map[string]string `protobuf:"bytes,2,rep,name=options,proto3" json:"options,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + SourceOptions map[string]string `protobuf:"bytes,2,rep,name=sourceOptions,proto3" json:"sourceOptions,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + JobOptions map[string]string `protobuf:"bytes,5,rep,name=jobOptions,proto3" json:"jobOptions,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` Entities []string `protobuf:"bytes,3,rep,name=entities,proto3" json:"entities,omitempty"` Schema *Schema `protobuf:"bytes,4,opt,name=schema,proto3" json:"schema,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` @@ -39,7 +34,7 @@ func (m *ImportSpec) Reset() { *m = ImportSpec{} } func (m *ImportSpec) String() string { return proto.CompactTextString(m) } func (*ImportSpec) ProtoMessage() {} func (*ImportSpec) Descriptor() ([]byte, []int) { - return fileDescriptor_ImportSpec_d4d069df5227ffcf, []int{0} + return fileDescriptor_ImportSpec_673bc4f248a91137, []int{0} } func (m *ImportSpec) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_ImportSpec.Unmarshal(m, b) @@ -66,9 +61,16 @@ func (m *ImportSpec) GetType() string { return "" } -func (m *ImportSpec) GetOptions() map[string]string { +func (m *ImportSpec) GetSourceOptions() map[string]string { if m != nil { - return m.Options + return m.SourceOptions + } + return nil +} + +func (m *ImportSpec) GetJobOptions() map[string]string { + if m != nil { + return m.JobOptions } return nil } @@ -89,6 +91,8 @@ func (m *ImportSpec) GetSchema() *Schema { type Schema struct { Fields []*Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"` + // the event timestamp to set per row. + // // Types that are valid to be assigned to Timestamp: // *Schema_TimestampColumn // *Schema_TimestampValue @@ -103,7 +107,7 @@ func (m *Schema) Reset() { *m = Schema{} } func (m *Schema) String() string { return proto.CompactTextString(m) } func (*Schema) ProtoMessage() {} func (*Schema) Descriptor() ([]byte, []int) { - return fileDescriptor_ImportSpec_d4d069df5227ffcf, []int{1} + return fileDescriptor_ImportSpec_673bc4f248a91137, []int{1} } func (m *Schema) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Schema.Unmarshal(m, b) @@ -256,7 +260,7 @@ func (m *Field) Reset() { *m = Field{} } func (m *Field) String() string { return proto.CompactTextString(m) } func (*Field) ProtoMessage() {} func (*Field) Descriptor() ([]byte, []int) { - return fileDescriptor_ImportSpec_d4d069df5227ffcf, []int{2} + return fileDescriptor_ImportSpec_673bc4f248a91137, []int{2} } func (m *Field) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Field.Unmarshal(m, b) @@ -292,41 +296,44 @@ func (m *Field) GetFeatureId() string { func init() { proto.RegisterType((*ImportSpec)(nil), "feast.specs.ImportSpec") - proto.RegisterMapType((map[string]string)(nil), "feast.specs.ImportSpec.OptionsEntry") + proto.RegisterMapType((map[string]string)(nil), "feast.specs.ImportSpec.JobOptionsEntry") + proto.RegisterMapType((map[string]string)(nil), "feast.specs.ImportSpec.SourceOptionsEntry") proto.RegisterType((*Schema)(nil), "feast.specs.Schema") proto.RegisterType((*Field)(nil), "feast.specs.Field") } func init() { - proto.RegisterFile("feast/specs/ImportSpec.proto", fileDescriptor_ImportSpec_d4d069df5227ffcf) -} - -var fileDescriptor_ImportSpec_d4d069df5227ffcf = []byte{ - // 413 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0x52, 0x5b, 0x6b, 0xd4, 0x40, - 0x14, 0x6e, 0x76, 0xbb, 0xa9, 0x7b, 0x22, 0xad, 0x8c, 0x82, 0x43, 0x28, 0x18, 0x16, 0x91, 0x50, - 0x61, 0x06, 0x2a, 0x88, 0xf6, 0xc1, 0x87, 0xf5, 0x42, 0xf7, 0x49, 0x49, 0xa5, 0x0f, 0xbe, 0x65, - 0x93, 0x93, 0x34, 0x36, 0xc9, 0x84, 0xcc, 0x44, 0xc8, 0x6f, 0xf5, 0xd9, 0xff, 0x21, 0x33, 0x93, - 0xcd, 0x66, 0xf7, 0xed, 0x5c, 0xbe, 0x73, 0xf9, 0xce, 0x77, 0xe0, 0x32, 0xc3, 0x58, 0x2a, 0x2e, - 0x1b, 0x4c, 0x24, 0xdf, 0x54, 0x8d, 0x68, 0xd5, 0x5d, 0x83, 0x09, 0x6b, 0x5a, 0xa1, 0x04, 0xf1, - 0x4c, 0x96, 0x99, 0xac, 0xff, 0xd2, 0x42, 0x55, 0xdf, 0xa0, 0xe4, 0xf7, 0x71, 0xd9, 0xa1, 0x45, - 0xf9, 0xaf, 0x72, 0x21, 0xf2, 0x12, 0xb9, 0xf1, 0xb6, 0x5d, 0xc6, 0x55, 0x51, 0xa1, 0x54, 0x71, - 0xd5, 0x58, 0xc0, 0xea, 0x9f, 0x03, 0xb0, 0xef, 0x4d, 0x08, 0x9c, 0xea, 0x26, 0xd4, 0x09, 0x9c, - 0x70, 0x19, 0x19, 0x9b, 0x7c, 0x82, 0x33, 0xd1, 0xa8, 0x42, 0xd4, 0x92, 0xce, 0x82, 0x79, 0xe8, - 0x5d, 0xbf, 0x66, 0x93, 0xd9, 0x6c, 0xb2, 0xd9, 0x77, 0x0b, 0xfb, 0x5a, 0xab, 0xb6, 0x8f, 0x76, - 0x45, 0xc4, 0x87, 0x27, 0x58, 0xab, 0x42, 0x15, 0x28, 0xe9, 0x3c, 0x98, 0x87, 0xcb, 0x68, 0xf4, - 0xc9, 0x5b, 0x70, 0x65, 0xf2, 0x80, 0x55, 0x4c, 0x4f, 0x03, 0x27, 0xf4, 0xae, 0x9f, 0x1f, 0xb4, - 0xbe, 0x33, 0xa9, 0x68, 0x80, 0xf8, 0x37, 0xf0, 0x74, 0x3a, 0x81, 0x3c, 0x83, 0xf9, 0x23, 0xf6, - 0xc3, 0xae, 0xda, 0x24, 0x2f, 0x60, 0xf1, 0x47, 0xb3, 0xa7, 0x33, 0x13, 0xb3, 0xce, 0xcd, 0xec, - 0x83, 0xb3, 0xfa, 0xeb, 0x80, 0x6b, 0xdb, 0x91, 0x2b, 0x70, 0xb3, 0x02, 0xcb, 0x54, 0x52, 0xc7, - 0xd0, 0x21, 0x07, 0x33, 0xbf, 0xe9, 0x54, 0x34, 0x20, 0xc8, 0x15, 0x5c, 0x8c, 0x17, 0xfb, 0x2c, - 0xca, 0xae, 0xaa, 0xe9, 0x42, 0xb7, 0xbe, 0x3d, 0x89, 0x8e, 0x13, 0xe4, 0x0b, 0x9c, 0x8f, 0x21, - 0xa3, 0x01, 0x75, 0x0d, 0x27, 0x9f, 0x59, 0x11, 0xd8, 0x4e, 0x04, 0xf6, 0x73, 0x07, 0xbb, 0x3d, - 0x89, 0x8e, 0x6a, 0xc8, 0x1b, 0x38, 0x37, 0xd7, 0xe9, 0x37, 0xe9, 0x30, 0xf0, 0xcc, 0x70, 0x39, - 0x8a, 0xae, 0x3d, 0x58, 0x8e, 0x95, 0xab, 0x8f, 0xb0, 0x30, 0x7b, 0x6b, 0xfd, 0xea, 0xb8, 0x1a, - 0xf5, 0xd3, 0x36, 0xb9, 0x84, 0x65, 0x86, 0xb1, 0xea, 0x5a, 0xdc, 0xa4, 0xc3, 0x61, 0xf6, 0x81, - 0xf5, 0x3d, 0x4c, 0x3f, 0x69, 0x7d, 0xb1, 0x97, 0xf3, 0x87, 0x5e, 0xf7, 0xd7, 0xfb, 0xbc, 0x50, - 0x0f, 0xdd, 0x96, 0x25, 0xa2, 0xe2, 0xb9, 0xf8, 0x8d, 0x8f, 0xdc, 0xfe, 0x9a, 0x21, 0x23, 0x79, - 0x8e, 0x35, 0xb6, 0xb1, 0xc2, 0x94, 0xe7, 0x82, 0x4f, 0x1e, 0x76, 0xeb, 0x1a, 0xc0, 0xbb, 0xff, - 0x01, 0x00, 0x00, 0xff, 0xff, 0xd1, 0x43, 0xb0, 0x30, 0xc6, 0x02, 0x00, 0x00, + proto.RegisterFile("feast/specs/ImportSpec.proto", fileDescriptor_ImportSpec_673bc4f248a91137) +} + +var fileDescriptor_ImportSpec_673bc4f248a91137 = []byte{ + // 440 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x53, 0x5d, 0x8f, 0x93, 0x40, + 0x14, 0x5d, 0xca, 0x16, 0xed, 0x6d, 0xdc, 0x9a, 0xab, 0x0f, 0x84, 0x6c, 0x62, 0xd3, 0x07, 0x6d, + 0x6a, 0x32, 0x93, 0xac, 0x89, 0x51, 0x13, 0x13, 0x53, 0xbf, 0xb6, 0xbe, 0xb8, 0xa1, 0x66, 0x1f, + 0x7c, 0xa3, 0x70, 0x61, 0xd9, 0x05, 0x86, 0x30, 0x83, 0x09, 0x3f, 0xc0, 0x5f, 0xe9, 0x9f, 0x31, + 0x0c, 0x94, 0x52, 0x8c, 0x0f, 0xfb, 0x36, 0xdc, 0x73, 0xee, 0xb9, 0x67, 0x2e, 0x67, 0xe0, 0x3c, + 0x24, 0x4f, 0x2a, 0x2e, 0x73, 0xf2, 0x25, 0xdf, 0xa4, 0xb9, 0x28, 0xd4, 0x36, 0x27, 0x9f, 0xe5, + 0x85, 0x50, 0x02, 0xa7, 0x1a, 0x65, 0x1a, 0x75, 0x9e, 0x45, 0x42, 0x44, 0x09, 0x71, 0x0d, 0xed, + 0xca, 0x90, 0xab, 0x38, 0x25, 0xa9, 0xbc, 0x34, 0x6f, 0xd8, 0x8b, 0xdf, 0x26, 0xc0, 0x41, 0x02, + 0x11, 0x4e, 0x55, 0x95, 0x93, 0x6d, 0xcc, 0x8d, 0xe5, 0xc4, 0xd5, 0x67, 0xbc, 0x82, 0x47, 0x52, + 0x94, 0x85, 0x4f, 0xdf, 0x73, 0x15, 0x8b, 0x4c, 0xda, 0xa3, 0xb9, 0xb9, 0x9c, 0x5e, 0xac, 0x58, + 0x6f, 0x10, 0xeb, 0xd9, 0xd8, 0xf6, 0xc9, 0x9f, 0x33, 0x55, 0x54, 0xee, 0xb1, 0x00, 0x7e, 0x05, + 0xb8, 0x15, 0xbb, 0xbd, 0xdc, 0x58, 0xcb, 0xbd, 0xf8, 0x9f, 0xdc, 0xb7, 0x8e, 0xd9, 0x68, 0xf5, + 0x5a, 0xd1, 0x81, 0x87, 0x94, 0xa9, 0x58, 0xc5, 0x24, 0x6d, 0x73, 0x6e, 0x2e, 0x27, 0x6e, 0xf7, + 0x8d, 0x2f, 0xc1, 0x92, 0xfe, 0x0d, 0xa5, 0x9e, 0x7d, 0x3a, 0x37, 0x96, 0xd3, 0x8b, 0x27, 0x47, + 0x03, 0xb6, 0x1a, 0x72, 0x5b, 0x8a, 0xf3, 0x01, 0xf0, 0x5f, 0xdb, 0xf8, 0x18, 0xcc, 0x3b, 0xaa, + 0xda, 0x65, 0xd4, 0x47, 0x7c, 0x0a, 0xe3, 0x5f, 0x5e, 0x52, 0x92, 0x3d, 0xd2, 0xb5, 0xe6, 0xe3, + 0xdd, 0xe8, 0x8d, 0xe1, 0xbc, 0x87, 0xd9, 0xc0, 0xe9, 0x7d, 0xda, 0x17, 0x7f, 0x0c, 0xb0, 0x1a, + 0x4f, 0xb8, 0x02, 0x2b, 0x8c, 0x29, 0x09, 0xa4, 0x6d, 0xe8, 0xcd, 0xe0, 0x91, 0xf1, 0x2f, 0x35, + 0xe4, 0xb6, 0x0c, 0x5c, 0xc1, 0xac, 0xfb, 0xa3, 0x1f, 0x45, 0x52, 0xa6, 0x99, 0x3d, 0xae, 0xa5, + 0x2f, 0x4f, 0xdc, 0x21, 0x80, 0x9f, 0xe0, 0xac, 0x2b, 0x5d, 0x6b, 0x17, 0x96, 0x5e, 0x8c, 0xc3, + 0x9a, 0x90, 0xb0, 0x7d, 0x48, 0xd8, 0x8f, 0x3d, 0xed, 0xf2, 0xc4, 0x1d, 0xf4, 0xe0, 0x73, 0x38, + 0xd3, 0x2b, 0xae, 0x36, 0x41, 0x3b, 0xf0, 0x81, 0xbe, 0xcb, 0xa0, 0xba, 0x9e, 0xc2, 0xa4, 0xeb, + 0x5c, 0xbc, 0x85, 0xb1, 0xf6, 0x5d, 0xe7, 0x2b, 0xf3, 0xd2, 0x2e, 0x5f, 0xf5, 0x19, 0xcf, 0x61, + 0x12, 0x92, 0xa7, 0xca, 0x82, 0x36, 0x41, 0xbb, 0x98, 0x43, 0x61, 0x7d, 0x0d, 0xfd, 0x40, 0xaf, + 0x67, 0x87, 0x64, 0x5c, 0xd5, 0x76, 0x7f, 0xbe, 0x8e, 0x62, 0x75, 0x53, 0xee, 0x98, 0x2f, 0x52, + 0x1e, 0x89, 0x5b, 0xba, 0xe3, 0xcd, 0xeb, 0xd0, 0x97, 0x91, 0x3c, 0xa2, 0x8c, 0x0a, 0x4f, 0x51, + 0xc0, 0x23, 0xc1, 0x7b, 0xef, 0x66, 0x67, 0x69, 0xc2, 0xab, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, + 0x91, 0xba, 0x56, 0x3c, 0x4d, 0x03, 0x00, 0x00, } diff --git a/protos/generated/go/feast/storage/BigTable.pb.go b/protos/generated/go/feast/storage/BigTable.pb.go index 21151396b28..e65b5e35fce 100644 --- a/protos/generated/go/feast/storage/BigTable.pb.go +++ b/protos/generated/go/feast/storage/BigTable.pb.go @@ -6,7 +6,6 @@ package storage // import "github.com/gojek/feast/protos/generated/go/feast/stor import proto "github.com/golang/protobuf/proto" import fmt "fmt" import math "math" -import _ "github.com/gojek/feast/protos/generated/go/feast/types" // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal @@ -34,7 +33,7 @@ func (m *BigTableRowKey) Reset() { *m = BigTableRowKey{} } func (m *BigTableRowKey) String() string { return proto.CompactTextString(m) } func (*BigTableRowKey) ProtoMessage() {} func (*BigTableRowKey) Descriptor() ([]byte, []int) { - return fileDescriptor_BigTable_546749cadd35d8e4, []int{0} + return fileDescriptor_BigTable_e25bdeffe0669ae7, []int{0} } func (m *BigTableRowKey) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BigTableRowKey.Unmarshal(m, b) @@ -80,23 +79,22 @@ func init() { } func init() { - proto.RegisterFile("feast/storage/BigTable.proto", fileDescriptor_BigTable_546749cadd35d8e4) + proto.RegisterFile("feast/storage/BigTable.proto", fileDescriptor_BigTable_e25bdeffe0669ae7) } -var fileDescriptor_BigTable_546749cadd35d8e4 = []byte{ - // 209 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0x90, 0xc1, 0x4b, 0x86, 0x30, - 0x18, 0x87, 0xf9, 0x0a, 0x02, 0x07, 0x7a, 0xd8, 0x25, 0x09, 0x89, 0xe8, 0x10, 0x9d, 0x7c, 0x89, - 0x2e, 0x9d, 0xbd, 0x4a, 0x20, 0x12, 0x11, 0xdd, 0x66, 0xbe, 0xce, 0xd5, 0x72, 0xb2, 0xbd, 0x5a, - 0xfb, 0xef, 0xa3, 0xa9, 0x94, 0xdf, 0xf5, 0xf7, 0x3c, 0x83, 0x67, 0x2f, 0xcb, 0x3a, 0x14, 0x8e, - 0xc0, 0x91, 0xb1, 0x42, 0x22, 0x14, 0x4a, 0x3e, 0x89, 0x46, 0x63, 0x3e, 0x5a, 0x43, 0x86, 0xc7, - 0x81, 0xe6, 0x2b, 0xbd, 0x38, 0x5f, 0x64, 0xf2, 0x23, 0x3a, 0x78, 0x16, 0x7a, 0x5a, 0xbd, 0xeb, - 0x99, 0x25, 0xdb, 0xcb, 0xda, 0x7c, 0x95, 0xe8, 0xf9, 0x25, 0x63, 0xae, 0x17, 0x77, 0x95, 0xc5, - 0x4e, 0x7d, 0xa7, 0x87, 0xab, 0xc3, 0x6d, 0x54, 0xff, 0x5b, 0x78, 0xc6, 0x22, 0x1c, 0x48, 0x91, - 0x2f, 0xd1, 0xa7, 0x27, 0x01, 0xff, 0x0d, 0xfc, 0x86, 0x25, 0x16, 0x67, 0xb4, 0x0e, 0xdb, 0x47, - 0xa5, 0xb5, 0x72, 0xe9, 0x69, 0x50, 0x8e, 0xd6, 0xe2, 0x85, 0xed, 0x0b, 0x8b, 0x78, 0xcb, 0xa8, - 0x7e, 0xbb, 0x5e, 0x1f, 0xa4, 0xa2, 0x7e, 0x6a, 0xf2, 0x37, 0xf3, 0x09, 0xd2, 0xbc, 0xe3, 0x07, - 0x2c, 0x3f, 0x08, 0xd5, 0x0e, 0x24, 0x0e, 0x68, 0x05, 0x61, 0x0b, 0xd2, 0xc0, 0xee, 0x10, 0xcd, - 0x59, 0x50, 0xee, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff, 0x43, 0x55, 0xcf, 0x14, 0x20, 0x01, 0x00, +var fileDescriptor_BigTable_e25bdeffe0669ae7 = []byte{ + // 193 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0x8f, 0xb1, 0x0b, 0x82, 0x40, + 0x18, 0x47, 0xb1, 0x20, 0xf0, 0x40, 0x87, 0x9b, 0x1c, 0x24, 0xa2, 0x21, 0x9a, 0x3c, 0xa2, 0xa5, + 0xd9, 0x55, 0x02, 0x91, 0x86, 0x68, 0x3b, 0xf3, 0xf3, 0xbc, 0x32, 0x2f, 0xee, 0xbe, 0x2c, 0xff, + 0xfb, 0xe8, 0x4a, 0xca, 0xd6, 0xdf, 0xef, 0x0d, 0xef, 0x91, 0xb0, 0x04, 0x6e, 0x90, 0x19, 0x54, + 0x9a, 0x0b, 0x60, 0xb1, 0x14, 0x3b, 0x9e, 0xd7, 0x10, 0x5d, 0xb5, 0x42, 0x45, 0x3d, 0xfb, 0x46, + 0x9f, 0x77, 0xde, 0x12, 0xbf, 0x07, 0x32, 0x75, 0x4f, 0xa0, 0xa3, 0x53, 0x42, 0x4c, 0xc5, 0x57, + 0xa9, 0x86, 0x52, 0x3e, 0x02, 0x67, 0xe6, 0x2c, 0xdd, 0xec, 0x67, 0xa1, 0x21, 0x71, 0xa1, 0x41, + 0x89, 0x5d, 0x02, 0x5d, 0x30, 0xb2, 0xf7, 0x77, 0xa0, 0x0b, 0xe2, 0x6b, 0x68, 0x41, 0x1b, 0x28, + 0xb6, 0xb2, 0xae, 0xa5, 0x09, 0xc6, 0x16, 0xf9, 0x5b, 0xe3, 0x3d, 0x19, 0x8a, 0xc4, 0x5e, 0xaf, + 0x91, 0xbe, 0x34, 0x0f, 0x1b, 0x21, 0xb1, 0xba, 0xe5, 0xd1, 0x51, 0x5d, 0x98, 0x50, 0x27, 0x38, + 0xb3, 0x77, 0x95, 0x8d, 0x30, 0x4c, 0x40, 0x03, 0x9a, 0x23, 0x14, 0x4c, 0x28, 0x36, 0xe8, 0xcd, + 0x27, 0x16, 0x59, 0x3f, 0x03, 0x00, 0x00, 0xff, 0xff, 0xc5, 0xd9, 0xf3, 0xb2, 0x07, 0x01, 0x00, 0x00, } diff --git a/protos/generated/go/feast/types/Feature.pb.go b/protos/generated/go/feast/types/Feature.pb.go index 5421909cfeb..51ed41fb54f 100644 --- a/protos/generated/go/feast/types/Feature.pb.go +++ b/protos/generated/go/feast/types/Feature.pb.go @@ -6,7 +6,6 @@ package types // import "github.com/gojek/feast/protos/generated/go/feast/types" import proto "github.com/golang/protobuf/proto" import fmt "fmt" import math "math" -import _ "github.com/golang/protobuf/ptypes/timestamp" // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal @@ -31,7 +30,7 @@ func (m *Feature) Reset() { *m = Feature{} } func (m *Feature) String() string { return proto.CompactTextString(m) } func (*Feature) ProtoMessage() {} func (*Feature) Descriptor() ([]byte, []int) { - return fileDescriptor_Feature_26989c9d13406a0c, []int{0} + return fileDescriptor_Feature_c2a5d99d9bf3ca9c, []int{0} } func (m *Feature) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Feature.Unmarshal(m, b) @@ -69,21 +68,19 @@ func init() { proto.RegisterType((*Feature)(nil), "feast.types.Feature") } -func init() { proto.RegisterFile("feast/types/Feature.proto", fileDescriptor_Feature_26989c9d13406a0c) } +func init() { proto.RegisterFile("feast/types/Feature.proto", fileDescriptor_Feature_c2a5d99d9bf3ca9c) } -var fileDescriptor_Feature_26989c9d13406a0c = []byte{ - // 197 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x4c, 0x8e, 0xb1, 0x6a, 0xc3, 0x30, - 0x14, 0x45, 0xb1, 0xa1, 0x2d, 0x95, 0x4b, 0x07, 0x2d, 0x75, 0xbd, 0xd4, 0x74, 0xf2, 0xa4, 0x07, - 0x09, 0xe4, 0x03, 0x1c, 0xc8, 0x1c, 0x1c, 0xc8, 0x90, 0x4d, 0x8e, 0x9f, 0x15, 0x25, 0x76, 0x64, - 0xac, 0xa7, 0x40, 0xfe, 0x3e, 0x58, 0xf6, 0xa0, 0xf9, 0x1e, 0xce, 0x3d, 0xec, 0xb7, 0x45, 0x69, - 0x09, 0xe8, 0x39, 0xa0, 0x85, 0x1d, 0x4a, 0x72, 0x23, 0x8a, 0x61, 0x34, 0x64, 0x78, 0xe2, 0x27, - 0xe1, 0xa7, 0xec, 0x4f, 0x19, 0xa3, 0x3a, 0x04, 0x3f, 0xd5, 0xae, 0x05, 0xd2, 0x3d, 0x5a, 0x92, - 0xfd, 0x30, 0xd3, 0xd9, 0x4f, 0x28, 0x3a, 0xca, 0xce, 0x2d, 0x9a, 0xff, 0x2d, 0xfb, 0x58, 0xbc, - 0xfc, 0x9b, 0xc5, 0xba, 0x49, 0xa3, 0x3c, 0x2a, 0x3e, 0xab, 0x58, 0x37, 0xbc, 0x60, 0x6f, 0x8f, - 0x89, 0x4c, 0xe3, 0x3c, 0x2a, 0x92, 0x15, 0x17, 0xc1, 0xa3, 0xf0, 0x8e, 0x6a, 0x06, 0xca, 0x03, - 0x0b, 0x6b, 0xca, 0xaf, 0xc5, 0xb8, 0x9f, 0x1e, 0x4e, 0x1b, 0xa5, 0xe9, 0xe2, 0x6a, 0x71, 0x36, - 0x3d, 0x28, 0x73, 0xc5, 0x1b, 0xcc, 0x2d, 0xfe, 0xdf, 0x82, 0xc2, 0x3b, 0x8e, 0x92, 0xb0, 0x01, - 0x65, 0x20, 0xa8, 0xac, 0xdf, 0x3d, 0xb0, 0x7e, 0x05, 0x00, 0x00, 0xff, 0xff, 0xbf, 0x23, 0xad, - 0x34, 0x04, 0x01, 0x00, 0x00, +var fileDescriptor_Feature_c2a5d99d9bf3ca9c = []byte{ + // 173 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x92, 0x4c, 0x4b, 0x4d, 0x2c, + 0x2e, 0xd1, 0x2f, 0xa9, 0x2c, 0x48, 0x2d, 0xd6, 0x77, 0x4b, 0x4d, 0x2c, 0x29, 0x2d, 0x4a, 0xd5, + 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0xe2, 0x06, 0x4b, 0xe9, 0x81, 0xa5, 0xa4, 0xc4, 0x91, 0xd5, + 0x85, 0x25, 0xe6, 0x94, 0x42, 0x55, 0x29, 0x39, 0x73, 0xb1, 0x43, 0xb5, 0x09, 0xf1, 0x71, 0x31, + 0x65, 0xa6, 0x48, 0x30, 0x2a, 0x30, 0x6a, 0x70, 0x06, 0x31, 0x65, 0xa6, 0x08, 0x69, 0x70, 0xb1, + 0x96, 0x81, 0x54, 0x4a, 0x30, 0x29, 0x30, 0x6a, 0x70, 0x1b, 0x09, 0xe9, 0x21, 0x19, 0xa8, 0x07, + 0x36, 0x23, 0x08, 0xa2, 0xc0, 0x29, 0x98, 0x0b, 0xd9, 0x32, 0x27, 0x1e, 0xa8, 0x89, 0x01, 0x20, + 0x1b, 0xa2, 0xcc, 0xd2, 0x33, 0x4b, 0x32, 0x4a, 0x93, 0xf4, 0x92, 0xf3, 0x73, 0xf5, 0xd3, 0xf3, + 0xb3, 0x52, 0xb3, 0xf5, 0x21, 0x6e, 0x01, 0xdb, 0x5f, 0xac, 0x9f, 0x9e, 0x9a, 0x97, 0x5a, 0x94, + 0x58, 0x92, 0x9a, 0xa2, 0x9f, 0x9e, 0xaf, 0x8f, 0xe4, 0xca, 0x24, 0x36, 0xb0, 0x02, 0x63, 0x40, + 0x00, 0x00, 0x00, 0xff, 0xff, 0x72, 0x5f, 0x2d, 0xd9, 0xe3, 0x00, 0x00, 0x00, } diff --git a/protos/generated/go/feast/types/FeatureRow.pb.go b/protos/generated/go/feast/types/FeatureRow.pb.go index d4e18a19754..3fb6bc72f9d 100644 --- a/protos/generated/go/feast/types/FeatureRow.pb.go +++ b/protos/generated/go/feast/types/FeatureRow.pb.go @@ -33,7 +33,7 @@ func (m *FeatureRowKey) Reset() { *m = FeatureRowKey{} } func (m *FeatureRowKey) String() string { return proto.CompactTextString(m) } func (*FeatureRowKey) ProtoMessage() {} func (*FeatureRowKey) Descriptor() ([]byte, []int) { - return fileDescriptor_FeatureRow_185cd01c950d2687, []int{0} + return fileDescriptor_FeatureRow_435f7325616f2983, []int{0} } func (m *FeatureRowKey) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FeatureRowKey.Unmarshal(m, b) @@ -96,7 +96,7 @@ func (m *FeatureRow) Reset() { *m = FeatureRow{} } func (m *FeatureRow) String() string { return proto.CompactTextString(m) } func (*FeatureRow) ProtoMessage() {} func (*FeatureRow) Descriptor() ([]byte, []int) { - return fileDescriptor_FeatureRow_185cd01c950d2687, []int{1} + return fileDescriptor_FeatureRow_435f7325616f2983, []int{1} } func (m *FeatureRow) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FeatureRow.Unmarshal(m, b) @@ -157,29 +157,29 @@ func init() { } func init() { - proto.RegisterFile("feast/types/FeatureRow.proto", fileDescriptor_FeatureRow_185cd01c950d2687) -} - -var fileDescriptor_FeatureRow_185cd01c950d2687 = []byte{ - // 311 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x92, 0x41, 0x4b, 0xc3, 0x40, - 0x10, 0x85, 0x49, 0xab, 0x62, 0x37, 0x58, 0x21, 0x08, 0xc6, 0xd2, 0x6a, 0xe8, 0x29, 0xa7, 0x5d, - 0x89, 0xe0, 0x55, 0x08, 0xa8, 0x87, 0x82, 0x48, 0x90, 0x1e, 0xbc, 0x6d, 0x74, 0xb2, 0x46, 0x93, - 0x6c, 0x48, 0x66, 0x95, 0x1c, 0xfd, 0x8b, 0xfe, 0x22, 0x71, 0x93, 0x34, 0xab, 0x08, 0x9e, 0xbd, - 0xce, 0xf7, 0x66, 0xe6, 0xbd, 0x61, 0xc8, 0x3c, 0x01, 0x5e, 0x23, 0xc3, 0xa6, 0x84, 0x9a, 0x5d, - 0x01, 0x47, 0x55, 0x41, 0x24, 0xdf, 0x68, 0x59, 0x49, 0x94, 0x8e, 0xad, 0x29, 0xd5, 0x74, 0x76, - 0x22, 0xa4, 0x14, 0x19, 0x30, 0x8d, 0x62, 0x95, 0x30, 0x4c, 0x73, 0xa8, 0x91, 0xe7, 0x65, 0xab, - 0x9e, 0x1d, 0xfd, 0x32, 0xab, 0x43, 0x0b, 0x13, 0x5d, 0x57, 0xbc, 0x50, 0x19, 0xaf, 0x52, 0x6c, - 0x3a, 0x7c, 0x68, 0xe2, 0x35, 0xcf, 0x54, 0xd7, 0xb7, 0xfc, 0xb0, 0xc8, 0xde, 0xe0, 0x6a, 0x05, - 0x8d, 0x33, 0x27, 0x13, 0x28, 0x30, 0xc5, 0x66, 0x05, 0x8d, 0x6b, 0x79, 0x96, 0x3f, 0x89, 0x86, - 0x82, 0x13, 0x92, 0x29, 0xbc, 0x42, 0x81, 0x77, 0xbd, 0x35, 0x77, 0xec, 0x59, 0xbe, 0x1d, 0xcc, - 0x68, 0x6b, 0x9e, 0xf6, 0xe6, 0xe9, 0x46, 0x11, 0xfd, 0xe8, 0x70, 0x8e, 0x09, 0x69, 0x07, 0xde, - 0xf0, 0x1c, 0xdc, 0x2d, 0xbd, 0xc2, 0xa8, 0x38, 0x17, 0xc4, 0x16, 0x43, 0x02, 0x77, 0xdb, 0xb3, - 0xfc, 0x69, 0xb0, 0xa0, 0xc6, 0xa9, 0xa8, 0x99, 0xf0, 0xb2, 0x50, 0x79, 0x64, 0x76, 0x2c, 0xdf, - 0x47, 0x84, 0x0c, 0xa1, 0xfe, 0x48, 0x74, 0x4a, 0x76, 0x93, 0x56, 0x5b, 0xbb, 0x23, 0x6f, 0xec, - 0xdb, 0xc1, 0xc1, 0xb7, 0x55, 0xfd, 0xa0, 0x8d, 0xea, 0x5f, 0xdc, 0x20, 0x5c, 0x13, 0xf3, 0xb7, - 0xc2, 0xfd, 0xe1, 0x1e, 0xb7, 0x5f, 0xee, 0xee, 0xcf, 0x45, 0x8a, 0x4f, 0x2a, 0xa6, 0x0f, 0x32, - 0x67, 0x42, 0x3e, 0xc3, 0x0b, 0x6b, 0x5f, 0x44, 0x7b, 0xaf, 0x99, 0x80, 0x02, 0x2a, 0x8e, 0xf0, - 0xc8, 0x84, 0x64, 0xc6, 0xf3, 0xc4, 0x3b, 0x5a, 0x70, 0xf6, 0x19, 0x00, 0x00, 0xff, 0xff, 0x3d, - 0x8f, 0x82, 0x0d, 0xd8, 0x02, 0x00, 0x00, + proto.RegisterFile("feast/types/FeatureRow.proto", fileDescriptor_FeatureRow_435f7325616f2983) +} + +var fileDescriptor_FeatureRow_435f7325616f2983 = []byte{ + // 305 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x92, 0x41, 0x4b, 0xfb, 0x40, + 0x10, 0xc5, 0x49, 0xfb, 0xff, 0x8b, 0x9d, 0x60, 0x85, 0xe0, 0x21, 0x96, 0x56, 0x43, 0x4f, 0x39, + 0xed, 0x4a, 0x04, 0xaf, 0x42, 0x40, 0x3d, 0x14, 0x44, 0x82, 0x78, 0xf0, 0xb6, 0xd1, 0xc9, 0x1a, + 0x6d, 0xb2, 0x21, 0x99, 0x28, 0x39, 0xfa, 0x15, 0xfd, 0x44, 0xe2, 0xae, 0x69, 0x56, 0x11, 0x3c, + 0x7b, 0x9d, 0xf7, 0x66, 0xe6, 0x37, 0xc3, 0x83, 0x79, 0x86, 0xa2, 0x21, 0x4e, 0x5d, 0x85, 0x0d, + 0x3f, 0x47, 0x41, 0x6d, 0x8d, 0x89, 0x7a, 0x61, 0x55, 0xad, 0x48, 0x79, 0xae, 0x56, 0x99, 0x56, + 0x67, 0x87, 0x52, 0x29, 0xb9, 0x46, 0xae, 0xa5, 0xb4, 0xcd, 0x38, 0xe5, 0x05, 0x36, 0x24, 0x8a, + 0xca, 0xb8, 0x67, 0xfb, 0x3f, 0xcc, 0xfa, 0x94, 0x16, 0xb6, 0x74, 0x51, 0x8b, 0xb2, 0x5d, 0x8b, + 0x3a, 0xa7, 0xce, 0xc8, 0xcb, 0x37, 0x07, 0x76, 0x86, 0xe5, 0x2b, 0xec, 0xbc, 0x39, 0x4c, 0xb0, + 0xa4, 0x9c, 0xba, 0x15, 0x76, 0xbe, 0x13, 0x38, 0xe1, 0x24, 0x19, 0x0a, 0x5e, 0x0c, 0x53, 0x7c, + 0xc6, 0x92, 0xae, 0x7b, 0x02, 0x7f, 0x1c, 0x38, 0xa1, 0x1b, 0xcd, 0x98, 0x61, 0x64, 0x3d, 0x23, + 0xdb, 0x38, 0x92, 0x6f, 0x1d, 0xde, 0x01, 0x80, 0x19, 0x78, 0x29, 0x0a, 0xf4, 0xff, 0xe9, 0x15, + 0x56, 0xc5, 0x3b, 0x05, 0x57, 0x0e, 0xa0, 0xfe, 0xff, 0xc0, 0x09, 0xa7, 0xd1, 0x82, 0x59, 0x1f, + 0x61, 0xf6, 0x21, 0x67, 0x65, 0x5b, 0x24, 0x76, 0xc7, 0xf2, 0x75, 0x04, 0x30, 0x1c, 0xf5, 0xcb, + 0x45, 0x47, 0xb0, 0x9d, 0x19, 0x6f, 0xe3, 0x8f, 0x82, 0x71, 0xe8, 0x46, 0x7b, 0x5f, 0x56, 0xf5, + 0x83, 0x36, 0xae, 0x3f, 0xf1, 0x83, 0xf8, 0x06, 0xec, 0x08, 0xc5, 0xbb, 0xc3, 0x3f, 0xae, 0x3e, + 0xe8, 0x6e, 0x4f, 0x64, 0x4e, 0x0f, 0x6d, 0xca, 0xee, 0x54, 0xc1, 0xa5, 0x7a, 0xc4, 0x27, 0x6e, + 0x82, 0xa2, 0xd9, 0x1b, 0x2e, 0xb1, 0xc4, 0x5a, 0x10, 0xde, 0x73, 0xa9, 0xb8, 0x15, 0xa1, 0x74, + 0x4b, 0x1b, 0x8e, 0xdf, 0x03, 0x00, 0x00, 0xff, 0xff, 0xb5, 0xfe, 0x14, 0xc9, 0xbf, 0x02, 0x00, + 0x00, } From a734340ce2984a5241a011baa1c0862f0165cce9 Mon Sep 17 00:00:00 2001 From: zhilingc Date: Mon, 11 Feb 2019 01:14:07 +0800 Subject: [PATCH 15/15] Update python sdk for new import spec --- sdk/python/feast/sdk/importer.py | 43 ++++++---- sdk/python/feast/specs/ImportSpec_pb2.py | 103 +++++++++++++++++------ sdk/python/tests/sdk/test_importer.py | 4 +- 3 files changed, 107 insertions(+), 43 deletions(-) diff --git a/sdk/python/feast/sdk/importer.py b/sdk/python/feast/sdk/importer.py index d6800c45d94..c7f31491b93 100644 --- a/sdk/python/feast/sdk/importer.py +++ b/sdk/python/feast/sdk/importer.py @@ -75,7 +75,8 @@ def entity(self): @classmethod def from_csv(cls, path, entity, granularity, owner, staging_location=None, id_column=None, feature_columns=None, timestamp_column=None, - timestamp_value=None, serving_store=None, warehouse_store=None): + timestamp_value=None, serving_store=None, warehouse_store=None, + job_options={}): """Creates an importer from a given csv dataset. This file can be either local or remote (in gcs). If it's a local file then staging_location must be determined. @@ -100,12 +101,13 @@ def from_csv(cls, path, entity, granularity, owner, staging_location=None, Serving store to write the features in this instance to. warehouse_store (feast.sdk.resources.feature.DataStore): Defaults to None. Warehouse store to write the features in this instance to. + job_options (dict): Defaults to empty dict. Additional job options. Returns: Importer: the importer for the dataset provided. """ - import_spec_options = {"format": "csv"} - import_spec_options["path"], require_staging = \ + source_options = {"format": "csv"} + source_options["path"], require_staging = \ _get_remote_location(path, staging_location) if is_gs_path(path): df = gcs_to_df(path) @@ -116,10 +118,10 @@ def from_csv(cls, path, entity, granularity, owner, staging_location=None, feature_columns, timestamp_column, timestamp_value, serving_store, warehouse_store, df) - iport_spec = _create_import("file", import_spec_options, entity, schema) + iport_spec = _create_import("file", source_options, job_options, entity, schema) props = (_properties("csv", len(df.index), require_staging, - import_spec_options["path"])) + source_options["path"])) specs = _specs(iport_spec, Entity(name=entity), features) return cls(specs, df, props) @@ -127,7 +129,8 @@ def from_csv(cls, path, entity, granularity, owner, staging_location=None, @classmethod def from_bq(cls, bq_path, entity, granularity, owner, limit=10, id_column=None, feature_columns=None, timestamp_column=None, - timestamp_value=None, serving_store=None, warehouse_store=None): + timestamp_value=None, serving_store=None, warehouse_store=None, + job_options={}): """Creates an importer from a given bigquery table. Args: @@ -151,6 +154,7 @@ def from_bq(cls, bq_path, entity, granularity, owner, limit=10, Serving store to write the features in this instance to. warehouse_store (feast.sdk.resources.feature.DataStore): Defaults to None. Warehouse store to write the features in this instance to. + job_options (dict): Defaults to empty dict. Additional job options. Returns: Importer: the importer for the dataset provided. @@ -162,7 +166,7 @@ def from_bq(cls, bq_path, entity, granularity, owner, limit=10, table_ref = dataset_ref.table(table_id) table = cli.get_table(table_ref) - import_spec_options = { + source_options = { "project": project, "dataset": dataset_id, "table": table_id @@ -173,8 +177,8 @@ def from_bq(cls, bq_path, entity, granularity, owner, limit=10, feature_columns, timestamp_column, timestamp_value, serving_store, warehouse_store, df) - iport_spec = _create_import("bigquery", import_spec_options, - entity, schema) + iport_spec = _create_import("bigquery", source_options, + job_options, entity, schema) props = _properties("bigquery", table.num_rows, False, None) specs = _specs(iport_spec, Entity(name=entity), features) @@ -183,7 +187,8 @@ def from_bq(cls, bq_path, entity, granularity, owner, limit=10, @classmethod def from_df(cls, df, entity, granularity, owner, staging_location, id_column=None, feature_columns=None, timestamp_column=None, - timestamp_value=None, serving_store=None, warehouse_store=None): + timestamp_value=None, serving_store=None, warehouse_store=None, + job_options={}): """Creates an importer from a given pandas dataframe. To import a file from a dataframe, the data will have to be staged. @@ -207,26 +212,28 @@ def from_df(cls, df, entity, granularity, owner, staging_location, Serving store to write the features in this instance to. warehouse_store (feast.sdk.resources.feature.DataStore): Defaults to None. Warehouse store to write the features in this instance to. + job_options (dict): Defaults to empty dict. Additional job options. Returns: Importer: the importer for the dataset provided. """ tmp_file_name = ("tmp_{}_{}.csv" .format(entity, int(round(time.time() * 1000)))) - import_spec_options = { + source_options = { "format": "csv" } - import_spec_options["path"], require_staging = ( + source_options["path"], require_staging = ( _get_remote_location(tmp_file_name, staging_location)) schema, features = \ _detect_schema_and_feature(entity, granularity, owner, id_column, feature_columns, timestamp_column, timestamp_value, serving_store, warehouse_store, df) - iport_spec = _create_import("file", import_spec_options, entity, schema) + iport_spec = _create_import("file", source_options, + job_options, entity, schema) props = _properties("dataframe", len(df.index), require_staging, - import_spec_options["path"]) + source_options["path"]) specs = _specs(iport_spec, Entity(name=entity), features) return cls(specs, df, props) @@ -434,12 +441,13 @@ def _create_feature(column, entity, granularity, owner, return feature -def _create_import(import_type, options, entity, schema): +def _create_import(import_type, source_options, job_options, entity, schema): """Create an import spec. Args: import_type (str): import type - options (dict): import spec options + source_options (dict): import spec source options + jobOptions (dict): import spec job options entity (str): entity schema (feast.specs.ImportSpec_pb2.Schema): schema of the file @@ -449,7 +457,8 @@ def _create_import(import_type, options, entity, schema): return ImportSpec( type=import_type, - options=options, + sourceOptions=source_options, + jobOptions=job_options, entities=[entity], schema=schema) diff --git a/sdk/python/feast/specs/ImportSpec_pb2.py b/sdk/python/feast/specs/ImportSpec_pb2.py index d87fdafa4ad..60e5b4f400a 100644 --- a/sdk/python/feast/specs/ImportSpec_pb2.py +++ b/sdk/python/feast/specs/ImportSpec_pb2.py @@ -20,29 +20,29 @@ package='feast.specs', syntax='proto3', serialized_options=_b('\n\013feast.specsB\017ImportSpecProtoZ6github.com/gojek/feast/protos/generated/go/feast/specs'), - serialized_pb=_b('\n\x1c\x66\x65\x61st/specs/ImportSpec.proto\x12\x0b\x66\x65\x61st.specs\x1a\x1fgoogle/protobuf/timestamp.proto\"\xb8\x01\n\nImportSpec\x12\x0c\n\x04type\x18\x01 \x01(\t\x12\x35\n\x07options\x18\x02 \x03(\x0b\x32$.feast.specs.ImportSpec.OptionsEntry\x12\x10\n\x08\x65ntities\x18\x03 \x03(\t\x12#\n\x06schema\x18\x04 \x01(\x0b\x32\x13.feast.specs.Schema\x1a.\n\x0cOptionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xa2\x01\n\x06Schema\x12\"\n\x06\x66ields\x18\x01 \x03(\x0b\x32\x12.feast.specs.Field\x12\x19\n\x0ftimestampColumn\x18\x05 \x01(\tH\x00\x12\x34\n\x0etimestampValue\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampH\x00\x12\x16\n\x0e\x65ntityIdColumn\x18\x07 \x01(\tB\x0b\n\ttimestamp\"(\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x11\n\tfeatureId\x18\x02 \x01(\tBV\n\x0b\x66\x65\x61st.specsB\x0fImportSpecProtoZ6github.com/gojek/feast/protos/generated/go/feast/specsb\x06proto3') + serialized_pb=_b('\n\x1c\x66\x65\x61st/specs/ImportSpec.proto\x12\x0b\x66\x65\x61st.specs\x1a\x1fgoogle/protobuf/timestamp.proto\"\xba\x02\n\nImportSpec\x12\x0c\n\x04type\x18\x01 \x01(\t\x12\x41\n\rsourceOptions\x18\x02 \x03(\x0b\x32*.feast.specs.ImportSpec.SourceOptionsEntry\x12;\n\njobOptions\x18\x05 \x03(\x0b\x32\'.feast.specs.ImportSpec.JobOptionsEntry\x12\x10\n\x08\x65ntities\x18\x03 \x03(\t\x12#\n\x06schema\x18\x04 \x01(\x0b\x32\x13.feast.specs.Schema\x1a\x34\n\x12SourceOptionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1a\x31\n\x0fJobOptionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xa2\x01\n\x06Schema\x12\"\n\x06\x66ields\x18\x01 \x03(\x0b\x32\x12.feast.specs.Field\x12\x19\n\x0ftimestampColumn\x18\x05 \x01(\tH\x00\x12\x34\n\x0etimestampValue\x18\x06 \x01(\x0b\x32\x1a.google.protobuf.TimestampH\x00\x12\x16\n\x0e\x65ntityIdColumn\x18\x07 \x01(\tB\x0b\n\ttimestamp\"(\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x11\n\tfeatureId\x18\x02 \x01(\tBV\n\x0b\x66\x65\x61st.specsB\x0fImportSpecProtoZ6github.com/gojek/feast/protos/generated/go/feast/specsb\x06proto3') , dependencies=[google_dot_protobuf_dot_timestamp__pb2.DESCRIPTOR,]) -_IMPORTSPEC_OPTIONSENTRY = _descriptor.Descriptor( - name='OptionsEntry', - full_name='feast.specs.ImportSpec.OptionsEntry', +_IMPORTSPEC_SOURCEOPTIONSENTRY = _descriptor.Descriptor( + name='SourceOptionsEntry', + full_name='feast.specs.ImportSpec.SourceOptionsEntry', filename=None, file=DESCRIPTOR, containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='key', full_name='feast.specs.ImportSpec.OptionsEntry.key', index=0, + name='key', full_name='feast.specs.ImportSpec.SourceOptionsEntry.key', index=0, number=1, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='value', full_name='feast.specs.ImportSpec.OptionsEntry.value', index=1, + name='value', full_name='feast.specs.ImportSpec.SourceOptionsEntry.value', index=1, number=2, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, @@ -60,8 +60,45 @@ extension_ranges=[], oneofs=[ ], - serialized_start=217, - serialized_end=263, + serialized_start=290, + serialized_end=342, +) + +_IMPORTSPEC_JOBOPTIONSENTRY = _descriptor.Descriptor( + name='JobOptionsEntry', + full_name='feast.specs.ImportSpec.JobOptionsEntry', + filename=None, + file=DESCRIPTOR, + containing_type=None, + fields=[ + _descriptor.FieldDescriptor( + name='key', full_name='feast.specs.ImportSpec.JobOptionsEntry.key', index=0, + number=1, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='value', full_name='feast.specs.ImportSpec.JobOptionsEntry.value', index=1, + number=2, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + ], + extensions=[ + ], + nested_types=[], + enum_types=[ + ], + serialized_options=_b('8\001'), + is_extendable=False, + syntax='proto3', + extension_ranges=[], + oneofs=[ + ], + serialized_start=344, + serialized_end=393, ) _IMPORTSPEC = _descriptor.Descriptor( @@ -79,21 +116,28 @@ is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='options', full_name='feast.specs.ImportSpec.options', index=1, + name='sourceOptions', full_name='feast.specs.ImportSpec.sourceOptions', index=1, number=2, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='entities', full_name='feast.specs.ImportSpec.entities', index=2, + name='jobOptions', full_name='feast.specs.ImportSpec.jobOptions', index=2, + number=5, type=11, cpp_type=10, label=3, + has_default_value=False, default_value=[], + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='entities', full_name='feast.specs.ImportSpec.entities', index=3, number=3, type=9, cpp_type=9, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='schema', full_name='feast.specs.ImportSpec.schema', index=3, + name='schema', full_name='feast.specs.ImportSpec.schema', index=4, number=4, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, @@ -102,7 +146,7 @@ ], extensions=[ ], - nested_types=[_IMPORTSPEC_OPTIONSENTRY, ], + nested_types=[_IMPORTSPEC_SOURCEOPTIONSENTRY, _IMPORTSPEC_JOBOPTIONSENTRY, ], enum_types=[ ], serialized_options=None, @@ -112,7 +156,7 @@ oneofs=[ ], serialized_start=79, - serialized_end=263, + serialized_end=393, ) @@ -166,8 +210,8 @@ name='timestamp', full_name='feast.specs.Schema.timestamp', index=0, containing_type=None, fields=[]), ], - serialized_start=266, - serialized_end=428, + serialized_start=396, + serialized_end=558, ) @@ -204,12 +248,14 @@ extension_ranges=[], oneofs=[ ], - serialized_start=430, - serialized_end=470, + serialized_start=560, + serialized_end=600, ) -_IMPORTSPEC_OPTIONSENTRY.containing_type = _IMPORTSPEC -_IMPORTSPEC.fields_by_name['options'].message_type = _IMPORTSPEC_OPTIONSENTRY +_IMPORTSPEC_SOURCEOPTIONSENTRY.containing_type = _IMPORTSPEC +_IMPORTSPEC_JOBOPTIONSENTRY.containing_type = _IMPORTSPEC +_IMPORTSPEC.fields_by_name['sourceOptions'].message_type = _IMPORTSPEC_SOURCEOPTIONSENTRY +_IMPORTSPEC.fields_by_name['jobOptions'].message_type = _IMPORTSPEC_JOBOPTIONSENTRY _IMPORTSPEC.fields_by_name['schema'].message_type = _SCHEMA _SCHEMA.fields_by_name['fields'].message_type = _FIELD _SCHEMA.fields_by_name['timestampValue'].message_type = google_dot_protobuf_dot_timestamp__pb2._TIMESTAMP @@ -226,10 +272,17 @@ ImportSpec = _reflection.GeneratedProtocolMessageType('ImportSpec', (_message.Message,), dict( - OptionsEntry = _reflection.GeneratedProtocolMessageType('OptionsEntry', (_message.Message,), dict( - DESCRIPTOR = _IMPORTSPEC_OPTIONSENTRY, + SourceOptionsEntry = _reflection.GeneratedProtocolMessageType('SourceOptionsEntry', (_message.Message,), dict( + DESCRIPTOR = _IMPORTSPEC_SOURCEOPTIONSENTRY, + __module__ = 'feast.specs.ImportSpec_pb2' + # @@protoc_insertion_point(class_scope:feast.specs.ImportSpec.SourceOptionsEntry) + )) + , + + JobOptionsEntry = _reflection.GeneratedProtocolMessageType('JobOptionsEntry', (_message.Message,), dict( + DESCRIPTOR = _IMPORTSPEC_JOBOPTIONSENTRY, __module__ = 'feast.specs.ImportSpec_pb2' - # @@protoc_insertion_point(class_scope:feast.specs.ImportSpec.OptionsEntry) + # @@protoc_insertion_point(class_scope:feast.specs.ImportSpec.JobOptionsEntry) )) , DESCRIPTOR = _IMPORTSPEC, @@ -237,7 +290,8 @@ # @@protoc_insertion_point(class_scope:feast.specs.ImportSpec) )) _sym_db.RegisterMessage(ImportSpec) -_sym_db.RegisterMessage(ImportSpec.OptionsEntry) +_sym_db.RegisterMessage(ImportSpec.SourceOptionsEntry) +_sym_db.RegisterMessage(ImportSpec.JobOptionsEntry) Schema = _reflection.GeneratedProtocolMessageType('Schema', (_message.Message,), dict( DESCRIPTOR = _SCHEMA, @@ -255,5 +309,6 @@ DESCRIPTOR._options = None -_IMPORTSPEC_OPTIONSENTRY._options = None +_IMPORTSPEC_SOURCEOPTIONSENTRY._options = None +_IMPORTSPEC_JOBOPTIONSENTRY._options = None # @@protoc_insertion_point(module_scope) diff --git a/sdk/python/tests/sdk/test_importer.py b/sdk/python/tests/sdk/test_importer.py index ba18deea371..c09cad0db87 100644 --- a/sdk/python/tests/sdk/test_importer.py +++ b/sdk/python/tests/sdk/test_importer.py @@ -151,7 +151,7 @@ def test_from_df(self): import_spec = importer.spec assert import_spec.type == "file" - assert import_spec.options == {"format" : "csv", "path" : importer.remote_path} + assert import_spec.sourceOptions == {"format" : "csv", "path" : importer.remote_path} assert import_spec.entities == ["driver"] schema = import_spec.schema @@ -185,7 +185,7 @@ def _validate_csv_importer(self, import_spec = importer.spec assert import_spec.type == "file" path = importer.remote_path if importer.require_staging else csv_path - assert import_spec.options == {"format" : "csv", "path" : path} + assert import_spec.sourceOptions == {"format" : "csv", "path" : path} assert import_spec.entities == [entity_name] schema = import_spec.schema