Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix BigQuerySink successful output to produce only once #858

Merged
merged 1 commit into from
Jul 3, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -218,14 +218,14 @@ public void process(ProcessContext c) {
@ProcessElement
public void process(ProcessContext c) {
CoGbkResult result = c.element().getValue();
boolean ready = result.getAll(successTag).iterator().hasNext();
if (!ready) {
return;
}

result
.getAll(successTag)
.forEach(
success ->
result
.getAll(inputTag)
.forEach(
rows -> rows.getFeatureRows().forEachRemaining(c::output)));
.getAll(inputTag)
.forEach(rows -> rows.getFeatureRows().forEachRemaining(c::output));
}
}));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import static feast.storage.common.testing.TestUtil.field;
import static feast.storage.connectors.bigquery.writer.FeatureSetSpecToTableSchema.*;
import static org.hamcrest.CoreMatchers.*;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.Assert.*;
import static org.mockito.Mockito.*;
import static org.mockito.MockitoAnnotations.initMocks;
Expand Down Expand Up @@ -464,7 +465,7 @@ public void featureRowCompressShouldPackAndUnpackSuccessfully() {
PCollection<FeatureRow> result =
p.apply(Create.of(inputWithNulls))
.apply("KV", ParDo.of(new ExtractKV()))
.apply(new CompactFeatureRows(1000))
.apply(new CompactFeatureRows(10000))
.apply("Flat", ParDo.of(new FlatMap()));

List<FeatureRow> inputWithoutNulls = dropNullFeature(input);
Expand All @@ -480,7 +481,15 @@ public void featureRowCompressShouldPackAndUnpackSuccessfully() {
.addAllFields(copyFieldsWithout(rowWithNull, "entity", "null_value"))
.build());

PAssert.that(result).containsInAnyOrder(inputWithoutNulls);
PAssert.that(result)
.satisfies(
actual -> {
List<FeatureRow> actualSorted = sortFeaturesByName(Lists.newArrayList(actual));
List<FeatureRow> expectedSorted = sortFeaturesByName(inputWithoutNulls);

assertThat(actualSorted, containsInAnyOrder(expectedSorted.toArray()));
return null;
});
p.run();
}

Expand All @@ -490,10 +499,7 @@ private List<FeatureRow> dropNullFeature(List<FeatureRow> input) {
r ->
FeatureRow.newBuilder()
.setFeatureSet(r.getFeatureSet())
.addAllFields(
r.getFieldsList().stream()
.filter(f -> !f.getName().equals("null_value"))
.collect(Collectors.toList()))
.addAllFields(copyFieldsWithout(r, "null_value"))
.build())
.collect(Collectors.toList());
}
Expand All @@ -505,6 +511,21 @@ private List<FieldProto.Field> copyFieldsWithout(FeatureRow row, String... excep
.collect(Collectors.toList());
}

public static List<FeatureRow> sortFeaturesByName(List<FeatureRow> rows) {
return rows.stream()
.map(
row -> {
List<FieldProto.Field> fieldsList = Lists.newArrayList(row.getFieldsList());
fieldsList.sort(Comparator.comparing(FieldProto.Field::getName));

return FeatureRow.newBuilder()
.setFeatureSet(row.getFeatureSet())
.addAllFields(fieldsList)
.build();
})
.collect(Collectors.toList());
}

public static class TableAnswer implements Answer<Table>, Serializable {
TableId tableId;
TableDefinition tableDefinition;
Expand Down