forked from feast-dev/feast
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Make redis key creation more determinisitic (feast-dev#380) (feast-de…
…v#471) * Make redis key creation more determinisitic (feast-dev#380) * Add documentation to RedisKey in Redis.proto Ensure entities are sorted by the name Co-authored-by: David Heryanto <david.heryanto@hotmail.com>
- Loading branch information
1 parent
0af0637
commit 85f398e
Showing
3 changed files
with
197 additions
and
5 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
183 changes: 183 additions & 0 deletions
183
ingestion/src/test/java/feast/store/serving/redis/FeatureRowToRedisMutationDoFnTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,183 @@ | ||
/* | ||
* SPDX-License-Identifier: Apache-2.0 | ||
* Copyright 2018-2020 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.store.serving.redis; | ||
|
||
import static org.junit.Assert.*; | ||
|
||
import com.google.protobuf.Timestamp; | ||
import feast.core.FeatureSetProto; | ||
import feast.core.FeatureSetProto.EntitySpec; | ||
import feast.core.FeatureSetProto.FeatureSetSpec; | ||
import feast.core.FeatureSetProto.FeatureSpec; | ||
import feast.storage.RedisProto.RedisKey; | ||
import feast.store.serving.redis.RedisCustomIO.RedisMutation; | ||
import feast.types.FeatureRowProto.FeatureRow; | ||
import feast.types.FieldProto.Field; | ||
import feast.types.ValueProto.Value; | ||
import feast.types.ValueProto.ValueType.Enum; | ||
import java.util.Arrays; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.Map; | ||
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.transforms.Create; | ||
import org.apache.beam.sdk.transforms.ParDo; | ||
import org.apache.beam.sdk.transforms.SerializableFunction; | ||
import org.apache.beam.sdk.values.PCollection; | ||
import org.junit.Rule; | ||
import org.junit.Test; | ||
|
||
public class FeatureRowToRedisMutationDoFnTest { | ||
|
||
@Rule public transient TestPipeline p = TestPipeline.create(); | ||
|
||
private FeatureSetProto.FeatureSet fs = | ||
FeatureSetProto.FeatureSet.newBuilder() | ||
.setSpec( | ||
FeatureSetSpec.newBuilder() | ||
.setName("feature_set") | ||
.setVersion(1) | ||
.addEntities( | ||
EntitySpec.newBuilder() | ||
.setName("entity_id_primary") | ||
.setValueType(Enum.INT32) | ||
.build()) | ||
.addEntities( | ||
EntitySpec.newBuilder() | ||
.setName("entity_id_secondary") | ||
.setValueType(Enum.STRING) | ||
.build()) | ||
.addFeatures( | ||
FeatureSpec.newBuilder() | ||
.setName("feature_1") | ||
.setValueType(Enum.STRING) | ||
.build()) | ||
.addFeatures( | ||
FeatureSpec.newBuilder() | ||
.setName("feature_2") | ||
.setValueType(Enum.INT64) | ||
.build())) | ||
.build(); | ||
|
||
@Test | ||
public void shouldConvertRowWithDuplicateEntitiesToValidKey() { | ||
Map<String, FeatureSetProto.FeatureSet> featureSets = new HashMap<>(); | ||
featureSets.put("feature_set", fs); | ||
|
||
FeatureRow offendingRow = | ||
FeatureRow.newBuilder() | ||
.setFeatureSet("feature_set") | ||
.setEventTimestamp(Timestamp.newBuilder().setSeconds(10)) | ||
.addFields( | ||
Field.newBuilder() | ||
.setName("entity_id_primary") | ||
.setValue(Value.newBuilder().setInt32Val(1))) | ||
.addFields( | ||
Field.newBuilder() | ||
.setName("entity_id_primary") | ||
.setValue(Value.newBuilder().setInt32Val(2))) | ||
.addFields( | ||
Field.newBuilder() | ||
.setName("entity_id_secondary") | ||
.setValue(Value.newBuilder().setStringVal("a"))) | ||
.build(); | ||
|
||
PCollection<RedisMutation> output = | ||
p.apply(Create.of(Collections.singletonList(offendingRow))) | ||
.setCoder(ProtoCoder.of(FeatureRow.class)) | ||
.apply(ParDo.of(new FeatureRowToRedisMutationDoFn(featureSets))); | ||
|
||
RedisKey expectedKey = | ||
RedisKey.newBuilder() | ||
.setFeatureSet("feature_set") | ||
.addEntities( | ||
Field.newBuilder() | ||
.setName("entity_id_primary") | ||
.setValue(Value.newBuilder().setInt32Val(1))) | ||
.addEntities( | ||
Field.newBuilder() | ||
.setName("entity_id_secondary") | ||
.setValue(Value.newBuilder().setStringVal("a"))) | ||
.build(); | ||
|
||
PAssert.that(output) | ||
.satisfies( | ||
(SerializableFunction<Iterable<RedisMutation>, Void>) | ||
input -> { | ||
input.forEach( | ||
rm -> { | ||
assert (Arrays.equals(rm.getKey(), expectedKey.toByteArray())); | ||
assert (Arrays.equals(rm.getValue(), offendingRow.toByteArray())); | ||
}); | ||
return null; | ||
}); | ||
p.run(); | ||
} | ||
|
||
@Test | ||
public void shouldConvertRowWithOutOfOrderEntitiesToValidKey() { | ||
Map<String, FeatureSetProto.FeatureSet> featureSets = new HashMap<>(); | ||
featureSets.put("feature_set", fs); | ||
|
||
FeatureRow offendingRow = | ||
FeatureRow.newBuilder() | ||
.setFeatureSet("feature_set") | ||
.setEventTimestamp(Timestamp.newBuilder().setSeconds(10)) | ||
.addFields( | ||
Field.newBuilder() | ||
.setName("entity_id_secondary") | ||
.setValue(Value.newBuilder().setStringVal("a"))) | ||
.addFields( | ||
Field.newBuilder() | ||
.setName("entity_id_primary") | ||
.setValue(Value.newBuilder().setInt32Val(1))) | ||
.build(); | ||
|
||
PCollection<RedisMutation> output = | ||
p.apply(Create.of(Collections.singletonList(offendingRow))) | ||
.setCoder(ProtoCoder.of(FeatureRow.class)) | ||
.apply(ParDo.of(new FeatureRowToRedisMutationDoFn(featureSets))); | ||
|
||
RedisKey expectedKey = | ||
RedisKey.newBuilder() | ||
.setFeatureSet("feature_set") | ||
.addEntities( | ||
Field.newBuilder() | ||
.setName("entity_id_primary") | ||
.setValue(Value.newBuilder().setInt32Val(1))) | ||
.addEntities( | ||
Field.newBuilder() | ||
.setName("entity_id_secondary") | ||
.setValue(Value.newBuilder().setStringVal("a"))) | ||
.build(); | ||
|
||
PAssert.that(output) | ||
.satisfies( | ||
(SerializableFunction<Iterable<RedisMutation>, Void>) | ||
input -> { | ||
input.forEach( | ||
rm -> { | ||
assert (Arrays.equals(rm.getKey(), expectedKey.toByteArray())); | ||
assert (Arrays.equals(rm.getValue(), offendingRow.toByteArray())); | ||
}); | ||
return null; | ||
}); | ||
p.run(); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters