From 3f44c433cb7c61fd9ce7d0d78137900e461f2759 Mon Sep 17 00:00:00 2001 From: smadarasmi Date: Wed, 27 Nov 2019 14:16:33 +0700 Subject: [PATCH 1/5] create cassandra store for registration and ingestion * Downgraded Guava to 25 * Beam 2.16 uses Cassandra 3.4.0 (So we cannot use Cassandra 4.x which shades Guava) * Cassandra 3.4.0 uses Guava version 16.0 but has a compatibility check to use a different class when we use version > 19.0. * Guava version 26 (version previously used) has breaking change to method used in compatibility check in Cassandra's dependency, hence version 25 * Using Cassandra's internal field 'writetime' to handle out of order arrivals. When older records where the primary key already exist in Cassandra are ingested, they are set as tombstones in Cassandra and ignored on retrieval. * Aware that this way of handling out of order arrival is specific to Cassandra, but until we have a general way to handle out of order arrivals we need to do it this way * Cassandra's object mapper requires stating table's name along with @Table annotation * table_name is still part of CassandraConfig for use in serving module * if user registers CassandraConfig with a different table name other than "feature_store", this will throw an exception --- core/pom.xml | 2 +- .../core/config/FeatureStreamConfig.java | 2 +- .../core/job/dataflow/DataflowJobManager.java | 2 +- .../core/job/direct/DirectJobRegistry.java | 2 +- .../job/direct/DirectRunnerJobManager.java | 2 +- .../main/java/feast/core/log/AuditLogger.java | 2 +- .../core/service/JobCoordinatorService.java | 2 +- .../feast/core/service/JobStatusService.java | 2 +- .../java/feast/core/util/TypeConversion.java | 2 +- .../feast/core/validators/MatchersTest.java | 4 +- ingestion/pom.xml | 13 + .../transform/CassandraMutationMapper.java | 60 +++++ .../CassandraMutationMapperFactory.java | 42 ++++ .../ingestion/transform/WriteToStore.java | 44 +++- .../java/feast/ingestion/utils/StoreUtil.java | 87 +++++++ .../java/feast/ingestion/utils/ValueUtil.java | 57 +++++ .../serving/cassandra/CassandraMutation.java | 121 +++++++++ .../FeatureRowToCassandraMutationDoFn.java | 85 +++++++ .../CassandraWriteToStoreITTest.java | 238 ++++++++++++++++++ .../util/CassandraStoreUtilITTest.java | 167 ++++++++++++ ...FeatureRowToCassandraMutationDoFnTest.java | 225 +++++++++++++++++ .../src/test/java/feast/test/TestUtil.java | 144 ++++++++++- .../embedded-store/LoadCassandra.cql | 8 + pom.xml | 2 +- protos/feast/core/Store.proto | 14 +- 25 files changed, 1302 insertions(+), 27 deletions(-) create mode 100644 ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapper.java create mode 100644 ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapperFactory.java create mode 100644 ingestion/src/main/java/feast/ingestion/utils/ValueUtil.java create mode 100644 ingestion/src/main/java/feast/store/serving/cassandra/CassandraMutation.java create mode 100644 ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java create mode 100644 ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java create mode 100644 ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java create mode 100644 ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java create mode 100644 ingestion/src/test/resources/embedded-store/LoadCassandra.cql diff --git a/core/pom.xml b/core/pom.xml index f6e4909260..d5b09292e8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -114,7 +114,7 @@ protobuf-java-util - + com.google.guava guava diff --git a/core/src/main/java/feast/core/config/FeatureStreamConfig.java b/core/src/main/java/feast/core/config/FeatureStreamConfig.java index ca8240d780..6d9a30f9e9 100644 --- a/core/src/main/java/feast/core/config/FeatureStreamConfig.java +++ b/core/src/main/java/feast/core/config/FeatureStreamConfig.java @@ -66,7 +66,7 @@ public Source getDefaultSource(FeastProperties feastProperties) { } catch (InterruptedException | ExecutionException e) { if (e.getCause().getClass().equals(TopicExistsException.class)) { log.warn( - Strings.lenientFormat( + String.format( "Unable to create topic %s in the feature stream, topic already exists, using existing topic.", topicName)); } else { diff --git a/core/src/main/java/feast/core/job/dataflow/DataflowJobManager.java b/core/src/main/java/feast/core/job/dataflow/DataflowJobManager.java index 4e4533c4c9..f19cf1a656 100644 --- a/core/src/main/java/feast/core/job/dataflow/DataflowJobManager.java +++ b/core/src/main/java/feast/core/job/dataflow/DataflowJobManager.java @@ -121,7 +121,7 @@ public void abortJob(String dataflowJobId) { } catch (Exception e) { log.error("Unable to drain job with id: {}, cause: {}", dataflowJobId, e.getMessage()); throw new RuntimeException( - Strings.lenientFormat("Unable to drain job with id: %s", dataflowJobId), e); + String.format("Unable to drain job with id: %s", dataflowJobId), e); } } diff --git a/core/src/main/java/feast/core/job/direct/DirectJobRegistry.java b/core/src/main/java/feast/core/job/direct/DirectJobRegistry.java index f7ded9fec7..8f6c87053f 100644 --- a/core/src/main/java/feast/core/job/direct/DirectJobRegistry.java +++ b/core/src/main/java/feast/core/job/direct/DirectJobRegistry.java @@ -41,7 +41,7 @@ public DirectJobRegistry() { public void add(DirectJob job) { if (jobs.containsKey(job.getJobId())) { throw new IllegalArgumentException( - Strings.lenientFormat("Job with id %s already exists and is running", job.getJobId())); + String.format("Job with id %s already exists and is running", job.getJobId())); } jobs.put(job.getJobId(), job); } diff --git a/core/src/main/java/feast/core/job/direct/DirectRunnerJobManager.java b/core/src/main/java/feast/core/job/direct/DirectRunnerJobManager.java index a09fd39495..85b8a95dd5 100644 --- a/core/src/main/java/feast/core/job/direct/DirectRunnerJobManager.java +++ b/core/src/main/java/feast/core/job/direct/DirectRunnerJobManager.java @@ -149,7 +149,7 @@ public void abortJob(String extId) { job.abort(); } catch (IOException e) { throw new RuntimeException( - Strings.lenientFormat("Unable to abort DirectRunner job %s", extId), e); + String.format("Unable to abort DirectRunner job %s", extId), e); } jobs.remove(extId); } diff --git a/core/src/main/java/feast/core/log/AuditLogger.java b/core/src/main/java/feast/core/log/AuditLogger.java index 5349b5548b..2c60307805 100644 --- a/core/src/main/java/feast/core/log/AuditLogger.java +++ b/core/src/main/java/feast/core/log/AuditLogger.java @@ -44,7 +44,7 @@ public static void log( map.put("resource", resource.toString()); map.put("id", id); map.put("action", action.toString()); - map.put("detail", Strings.lenientFormat(detail, args)); + map.put("detail", String.format(detail, args)); ObjectMessage msg = new ObjectMessage(map); log.log(AUDIT_LEVEL, msg); diff --git a/core/src/main/java/feast/core/service/JobCoordinatorService.java b/core/src/main/java/feast/core/service/JobCoordinatorService.java index c56531a9da..b5c9fc6c1b 100644 --- a/core/src/main/java/feast/core/service/JobCoordinatorService.java +++ b/core/src/main/java/feast/core/service/JobCoordinatorService.java @@ -179,7 +179,7 @@ private JobInfo updateJob( public void abortJob(String id) { Optional jobOptional = jobInfoRepository.findById(id); if (!jobOptional.isPresent()) { - throw new RetrievalException(Strings.lenientFormat("Unable to retrieve job with id %s", id)); + throw new RetrievalException(String.format("Unable to retrieve job with id %s", id)); } JobInfo job = jobOptional.get(); if (JobStatus.getTerminalState().contains(job.getStatus())) { diff --git a/core/src/main/java/feast/core/service/JobStatusService.java b/core/src/main/java/feast/core/service/JobStatusService.java index db6cd41ee8..26d81647fa 100644 --- a/core/src/main/java/feast/core/service/JobStatusService.java +++ b/core/src/main/java/feast/core/service/JobStatusService.java @@ -66,7 +66,7 @@ public class JobStatusService { // public JobDetail getJob(String id) { // Optional job = jobInfoRepository.findById(id); // if (!job.isPresent()) { - // throw new RetrievalException(Strings.lenientFormat("Unable to retrieve job with id %s", + // throw new RetrievalException(String.format("Unable to retrieve job with id %s", // id)); // } // JobDetail.Builder jobDetailBuilder = job.get().getJobDetail().toBuilder(); diff --git a/core/src/main/java/feast/core/util/TypeConversion.java b/core/src/main/java/feast/core/util/TypeConversion.java index e01a551135..a7dd2b0d2a 100644 --- a/core/src/main/java/feast/core/util/TypeConversion.java +++ b/core/src/main/java/feast/core/util/TypeConversion.java @@ -85,7 +85,7 @@ public static String convertMapToJsonString(Map map) { public static String[] convertMapToArgs(Map map) { List args = new ArrayList<>(); for (Entry arg : map.entrySet()) { - args.add(Strings.lenientFormat("--%s=%s", arg.getKey(), arg.getValue())); + args.add(String.format("--%s=%s", arg.getKey(), arg.getValue())); } return args.toArray(new String[] {}); } diff --git a/core/src/test/java/feast/core/validators/MatchersTest.java b/core/src/test/java/feast/core/validators/MatchersTest.java index 774e58c7a8..13c9e006a4 100644 --- a/core/src/test/java/feast/core/validators/MatchersTest.java +++ b/core/src/test/java/feast/core/validators/MatchersTest.java @@ -43,7 +43,7 @@ public void checkUpperSnakeCaseShouldPassForLegitUpperSnakeCaseWithNumbers() { public void checkUpperSnakeCaseShouldThrowIllegalArgumentExceptionWithFieldForInvalidString() { exception.expect(IllegalArgumentException.class); exception.expectMessage( - Strings.lenientFormat( + String.format( "invalid value for field %s: %s", "someField", "argument must be in upper snake case, and cannot include any special characters.")); @@ -61,7 +61,7 @@ public void checkLowerSnakeCaseShouldPassForLegitLowerSnakeCase() { public void checkLowerSnakeCaseShouldThrowIllegalArgumentExceptionWithFieldForInvalidString() { exception.expect(IllegalArgumentException.class); exception.expectMessage( - Strings.lenientFormat( + String.format( "invalid value for field %s: %s", "someField", "argument must be in lower snake case, and cannot include any special characters.")); diff --git a/ingestion/pom.xml b/ingestion/pom.xml index eb89233518..72ac60578b 100644 --- a/ingestion/pom.xml +++ b/ingestion/pom.xml @@ -213,6 +213,12 @@ ${org.apache.beam.version} + + org.apache.beam + beam-sdks-java-io-cassandra + ${org.apache.beam.version} + + redis.clients jedis @@ -245,6 +251,13 @@ test + + org.cassandraunit + cassandra-unit-shaded + 3.11.2.0 + test + + com.google.guava guava diff --git a/ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapper.java b/ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapper.java new file mode 100644 index 0000000000..b1e5c4f0ce --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapper.java @@ -0,0 +1,60 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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 com.datastax.driver.core.ResultSet; +import com.datastax.driver.mapping.Mapper.Option; +import feast.store.serving.cassandra.CassandraMutation; +import java.io.Serializable; +import java.util.Iterator; +import java.util.concurrent.Future; +import org.apache.beam.sdk.io.cassandra.Mapper; + +/** A {@link Mapper} that supports writing {@code CassandraMutation}s with the Beam Cassandra IO. */ +public class CassandraMutationMapper implements Mapper, Serializable { + + private com.datastax.driver.mapping.Mapper mapper; + + CassandraMutationMapper(com.datastax.driver.mapping.Mapper mapper) { + this.mapper = mapper; + } + + @Override + public Iterator map(ResultSet resultSet) { + throw new UnsupportedOperationException("Only supports write operations"); + } + + @Override + public Future deleteAsync(CassandraMutation entityClass) { + throw new UnsupportedOperationException("Only supports write operations"); + } + + /** + * Saves records to Cassandra with: - Cassandra's internal write time set to the timestamp of the + * record. Cassandra will not override an existing record with the same partition key if the write + * time is older - Expiration of the record + * + * @param entityClass Cassandra's object mapper + */ + @Override + public Future saveAsync(CassandraMutation entityClass) { + return mapper.saveAsync( + entityClass, + Option.timestamp(entityClass.getWriteTime()), + Option.ttl(entityClass.getTtl())); + } +} diff --git a/ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapperFactory.java b/ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapperFactory.java new file mode 100644 index 0000000000..9f34465099 --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/transform/CassandraMutationMapperFactory.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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 com.datastax.driver.core.Session; +import com.datastax.driver.mapping.MappingManager; +import feast.store.serving.cassandra.CassandraMutation; +import org.apache.beam.sdk.io.cassandra.Mapper; +import org.apache.beam.sdk.transforms.SerializableFunction; + +public class CassandraMutationMapperFactory implements SerializableFunction { + + private transient MappingManager mappingManager; + private Class entityClass; + + public CassandraMutationMapperFactory(Class entityClass) { + this.entityClass = entityClass; + } + + @Override + public Mapper apply(Session session) { + if (mappingManager == null) { + this.mappingManager = new MappingManager(session); + } + + return new CassandraMutationMapper(mappingManager.mapper(entityClass)); + } +} diff --git a/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java b/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java index 6f697f1c6f..c89b95bfb2 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java +++ b/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java @@ -16,12 +16,14 @@ */ package feast.ingestion.transform; +import com.datastax.driver.core.Session; import com.google.api.services.bigquery.model.TableDataInsertAllResponse.InsertErrors; import com.google.api.services.bigquery.model.TableRow; import com.google.auto.value.AutoValue; import feast.core.FeatureSetProto.FeatureSetSpec; import feast.core.StoreProto.Store; import feast.core.StoreProto.Store.BigQueryConfig; +import feast.core.StoreProto.Store.CassandraConfig; import feast.core.StoreProto.Store.RedisConfig; import feast.core.StoreProto.Store.StoreType; import feast.ingestion.options.ImportOptions; @@ -29,11 +31,16 @@ import feast.ingestion.values.FailedElement; import feast.store.serving.bigquery.FeatureRowToTableRow; import feast.store.serving.bigquery.GetTableDestination; +import feast.store.serving.cassandra.CassandraMutation; +import feast.store.serving.cassandra.FeatureRowToCassandraMutationDoFn; import feast.store.serving.redis.FeatureRowToRedisMutationDoFn; import feast.store.serving.redis.RedisCustomIO; import feast.types.FeatureRowProto.FeatureRow; import java.io.IOException; +import java.util.Arrays; import java.util.Map; +import org.apache.beam.sdk.io.cassandra.CassandraIO; +import org.apache.beam.sdk.io.cassandra.Mapper; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method; @@ -47,10 +54,10 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptors; -import org.apache.beam.sdk.values.ValueInSingleWindow; import org.slf4j.Logger; @AutoValue @@ -61,8 +68,8 @@ public abstract class WriteToStore extends PTransform, P public static final String METRIC_NAMESPACE = "WriteToStore"; public static final String ELEMENTS_WRITTEN_METRIC = "elements_written"; - private static final Counter elementsWritten = Metrics - .counter(METRIC_NAMESPACE, ELEMENTS_WRITTEN_METRIC); + private static final Counter elementsWritten = + Metrics.counter(METRIC_NAMESPACE, ELEMENTS_WRITTEN_METRIC); public abstract Store getStore(); @@ -146,16 +153,37 @@ public void processElement(ProcessContext context) { .build()); } break; + case CASSANDRA: + CassandraConfig cassandraConfig = getStore().getCassandraConfig(); + SerializableFunction mapperFactory = + new CassandraMutationMapperFactory(CassandraMutation.class); + input + .apply( + "Create CassandraMutation from FeatureRow", + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + getFeatureSetSpecs(), cassandraConfig.getDefaultTtl()))) + .apply( + CassandraIO.write() + .withHosts(Arrays.asList(cassandraConfig.getBootstrapHosts().split(","))) + .withPort(cassandraConfig.getPort()) + .withKeyspace(cassandraConfig.getKeyspace()) + .withEntity(CassandraMutation.class) + .withMapperFactoryFn(mapperFactory)); + break; default: log.error("Store type '{}' is not supported. No Feature Row will be written.", storeType); break; } - input.apply("IncrementWriteToStoreElementsWrittenCounter", - MapElements.into(TypeDescriptors.booleans()).via((FeatureRow row) -> { - elementsWritten.inc(); - return true; - })); + input.apply( + "IncrementWriteToStoreElementsWrittenCounter", + MapElements.into(TypeDescriptors.booleans()) + .via( + (FeatureRow row) -> { + elementsWritten.inc(); + return true; + })); return PDone.in(input.getPipeline()); } diff --git a/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java b/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java index 5ceb8bd2f9..b92696c66d 100644 --- a/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java +++ b/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java @@ -18,6 +18,14 @@ import static feast.types.ValueProto.ValueType; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.KeyspaceMetadata; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.schemabuilder.Create; +import com.datastax.driver.core.schemabuilder.KeyspaceOptions; +import com.datastax.driver.core.schemabuilder.SchemaBuilder; +import com.datastax.driver.mapping.MappingManager; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryOptions; import com.google.cloud.bigquery.DatasetId; @@ -39,13 +47,19 @@ import feast.core.FeatureSetProto.FeatureSetSpec; import feast.core.FeatureSetProto.FeatureSpec; import feast.core.StoreProto.Store; +import feast.core.StoreProto.Store.CassandraConfig; import feast.core.StoreProto.Store.RedisConfig; import feast.core.StoreProto.Store.StoreType; +import feast.store.serving.cassandra.CassandraMutation; import feast.types.ValueProto.ValueType.Enum; +import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; +import java.util.stream.Collectors; import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import redis.clients.jedis.JedisPool; @@ -112,6 +126,9 @@ public static void setupStore(Store store, FeatureSetSpec featureSetSpec) { store.getBigqueryConfig().getDatasetId(), BigQueryOptions.getDefaultInstance().getService()); break; + case CASSANDRA: + StoreUtil.setupCassandra(store.getCassandraConfig()); + break; default: log.warn("Store type '{}' is unsupported", storeType); break; @@ -246,4 +263,74 @@ public static void checkRedisConnection(RedisConfig redisConfig) { } jedisPool.close(); } + + /** + * Ensures Cassandra is accessible, else throw a RuntimeException. Creates Cassandra keyspace and + * table if it does not already exist + * + * @param cassandraConfig Please refer to feast.core.Store proto + */ + public static void setupCassandra(CassandraConfig cassandraConfig) { + List contactPoints = + Arrays.stream(cassandraConfig.getBootstrapHosts().split(",")) + .map(host -> new InetSocketAddress(host, cassandraConfig.getPort())) + .collect(Collectors.toList()); + Cluster cluster = Cluster.builder().addContactPointsWithPorts(contactPoints).build(); + Session session; + + try { + String keyspace = cassandraConfig.getKeyspace(); + KeyspaceMetadata keyspaceMetadata = cluster.getMetadata().getKeyspace(keyspace); + if (keyspaceMetadata == null) { + log.info("Creating keyspace '{}'", keyspace); + Map replicationOptions = + cassandraConfig.getReplicationOptionsMap().entrySet().stream() + .collect(Collectors.toMap(Entry::getKey, Entry::getValue)); + KeyspaceOptions createKeyspace = + SchemaBuilder.createKeyspace(keyspace) + .ifNotExists() + .with() + .replication(replicationOptions); + session = cluster.newSession(); + session.execute(createKeyspace); + } + + session = cluster.connect(keyspace); + // Currently no support for creating table from entity mapper: + // https://datastax-oss.atlassian.net/browse/JAVA-569 + Create createTable = + SchemaBuilder.createTable(keyspace, cassandraConfig.getTableName()) + .ifNotExists() + .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) + .addClusteringColumn(CassandraMutation.FEATURE, DataType.text()) + .addStaticColumn(CassandraMutation.VALUE, DataType.blob()); + log.info("Create Cassandra table if not exists.."); + session.execute(createTable); + + validateCassandraTable(session); + + session.close(); + } catch (RuntimeException e) { + throw new RuntimeException( + String.format( + "Failed to connect to Cassandra at bootstrap hosts: '%s' port: '%s'. Please check that your Cassandra is running and accessible from Feast.", + contactPoints.stream() + .map(InetSocketAddress::getHostName) + .collect(Collectors.joining(",")), + cassandraConfig.getPort()), + e); + } + cluster.close(); + } + + private static void validateCassandraTable(Session session) { + try { + new MappingManager(session).mapper(CassandraMutation.class).getTableMetadata(); + } catch (RuntimeException e) { + throw new RuntimeException( + String.format( + "Table created does not match the datastax object mapper: %s", + CassandraMutation.class.getSimpleName())); + } + } } diff --git a/ingestion/src/main/java/feast/ingestion/utils/ValueUtil.java b/ingestion/src/main/java/feast/ingestion/utils/ValueUtil.java new file mode 100644 index 0000000000..87a327e772 --- /dev/null +++ b/ingestion/src/main/java/feast/ingestion/utils/ValueUtil.java @@ -0,0 +1,57 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.utils; + +import feast.types.ValueProto.Value; + +/** + * Utility class for converting {@link Value} of different types to a string for storing as key in + * data stores + */ +public class ValueUtil { + + public static String toString(Value value) { + String strValue; + switch (value.getValCase()) { + case BYTES_VAL: + strValue = value.getBytesVal().toString(); + break; + case STRING_VAL: + strValue = value.getStringVal(); + break; + case INT32_VAL: + strValue = String.valueOf(value.getInt32Val()); + break; + case INT64_VAL: + strValue = String.valueOf(value.getInt64Val()); + break; + case DOUBLE_VAL: + strValue = String.valueOf(value.getDoubleVal()); + break; + case FLOAT_VAL: + strValue = String.valueOf(value.getFloatVal()); + break; + case BOOL_VAL: + strValue = String.valueOf(value.getBoolVal()); + break; + default: + throw new IllegalArgumentException( + String.format("toString method not supported for type %s", value.getValCase())); + } + return strValue; + } +} diff --git a/ingestion/src/main/java/feast/store/serving/cassandra/CassandraMutation.java b/ingestion/src/main/java/feast/store/serving/cassandra/CassandraMutation.java new file mode 100644 index 0000000000..23bbb9c3b3 --- /dev/null +++ b/ingestion/src/main/java/feast/store/serving/cassandra/CassandraMutation.java @@ -0,0 +1,121 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.cassandra; + +import com.datastax.driver.mapping.annotations.ClusteringColumn; +import com.datastax.driver.mapping.annotations.Computed; +import com.datastax.driver.mapping.annotations.PartitionKey; +import com.datastax.driver.mapping.annotations.Table; +import feast.core.FeatureSetProto.EntitySpec; +import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.ingestion.utils.ValueUtil; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.FieldProto.Field; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; + +/** + * Cassandra's object mapper that handles basic CRUD operations in Cassandra tables More info: + * https://docs.datastax.com/en/developer/java-driver/3.1/manual/object_mapper/ + */ +@DefaultCoder(value = AvroCoder.class) +@Table(name = "feature_store") +public final class CassandraMutation implements Serializable { + + public static final String ENTITIES = "entities"; + public static final String FEATURE = "feature"; + public static final String VALUE = "value"; + + @PartitionKey private final String entities; + + @ClusteringColumn private final String feature; + + private final ByteBuffer value; + + @Computed(value = "writetime(value)") + private final long writeTime; + + @Computed(value = "ttl(value)") + private final int ttl; + + // NoArgs constructor is needed when using Beam's CassandraIO withEntity and specifying this + // class, + // it looks for an init() method + CassandraMutation() { + this.entities = null; + this.feature = null; + this.value = null; + this.writeTime = 0; + this.ttl = 0; + } + + CassandraMutation(String entities, String feature, ByteBuffer value, long writeTime, int ttl) { + this.entities = entities; + this.feature = feature; + this.value = value; + this.writeTime = writeTime; + this.ttl = ttl; + } + + public long getWriteTime() { + return writeTime; + } + + public int getTtl() { + return ttl; + } + + static String keyFromFeatureRow(FeatureSetSpec featureSetSpec, FeatureRow featureRow) { + Set entityNames = + featureSetSpec.getEntitiesList().stream() + .map(EntitySpec::getName) + .collect(Collectors.toSet()); + List entities = new ArrayList<>(); + for (Field field : featureRow.getFieldsList()) { + if (entityNames.contains(field.getName())) { + entities.add(field); + } + } + return featureRow.getFeatureSet() + + ":" + + entities.stream() + .map(f -> f.getName() + "=" + ValueUtil.toString(f.getValue())) + .collect(Collectors.joining("|")); + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + if (o instanceof CassandraMutation) { + CassandraMutation that = (CassandraMutation) o; + return this.entities.equals(that.entities) + && this.feature.equals(that.feature) + && this.value.equals(that.value) + && this.writeTime == that.writeTime + && this.ttl == that.ttl; + } + return false; + } +} diff --git a/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java b/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java new file mode 100644 index 0000000000..e416fa9b5a --- /dev/null +++ b/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java @@ -0,0 +1,85 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.cassandra; + +import com.google.protobuf.Duration; +import com.google.protobuf.util.Timestamps; +import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.core.FeatureSetProto.FeatureSpec; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.FieldProto.Field; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.beam.sdk.transforms.DoFn; +import org.slf4j.Logger; + +public class FeatureRowToCassandraMutationDoFn extends DoFn { + + private static final Logger log = + org.slf4j.LoggerFactory.getLogger(FeatureRowToCassandraMutationDoFn.class); + private Map featureSetSpecs; + private Map maxAges; + + public FeatureRowToCassandraMutationDoFn(Map specs, Duration defaultTtl) { + this.featureSetSpecs = specs; + this.maxAges = new HashMap<>(); + for (FeatureSetSpec spec : specs.values()) { + String featureSetName = spec.getName() + ":" + spec.getVersion(); + if (spec.getMaxAge() != null && spec.getMaxAge().getSeconds() > 0) { + maxAges.put(featureSetName, Math.toIntExact(spec.getMaxAge().getSeconds())); + } else { + maxAges.put(featureSetName, Math.toIntExact(defaultTtl.getSeconds())); + } + } + } + + /** Output a Cassandra mutation object for every feature in the feature row. */ + @ProcessElement + public void processElement(ProcessContext context) { + FeatureRow featureRow = context.element(); + try { + FeatureSetSpec featureSetSpec = featureSetSpecs.get(featureRow.getFeatureSet()); + Set featureNames = + featureSetSpec.getFeaturesList().stream() + .map(FeatureSpec::getName) + .collect(Collectors.toSet()); + String key = CassandraMutation.keyFromFeatureRow(featureSetSpec, featureRow); + + Collection mutations = new ArrayList<>(); + for (Field field : featureRow.getFieldsList()) { + if (featureNames.contains(field.getName())) { + mutations.add( + new CassandraMutation( + key, + field.getName(), + ByteBuffer.wrap(field.getValue().toByteArray()), + Timestamps.toMicros(featureRow.getEventTimestamp()), + maxAges.get(featureRow.getFeatureSet()))); + } + } + + mutations.forEach(context::output); + } catch (Exception e) { + log.error(e.getMessage(), e); + } + } +} diff --git a/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java b/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java new file mode 100644 index 0000000000..6a94502b94 --- /dev/null +++ b/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java @@ -0,0 +1,238 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.google.protobuf.InvalidProtocolBufferException; +import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.core.StoreProto.Store; +import feast.core.StoreProto.Store.CassandraConfig; +import feast.core.StoreProto.Store.StoreType; +import feast.test.TestUtil; +import feast.test.TestUtil.LocalCassandra; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.FieldProto.Field; +import feast.types.ValueProto.Value; +import feast.types.ValueProto.ValueType.Enum; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.apache.thrift.transport.TTransportException; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +public class CassandraWriteToStoreITTest implements Serializable { + private FeatureSetSpec featureSetSpec; + private FeatureRow row; + + class FakeCassandraWriteToStore extends WriteToStore { + + private FeatureSetSpec featureSetSpec; + + FakeCassandraWriteToStore(FeatureSetSpec featureSetSpec) { + this.featureSetSpec = featureSetSpec; + } + + @Override + public Store getStore() { + return Store.newBuilder() + .setType(StoreType.CASSANDRA) + .setName("SERVING") + .setCassandraConfig( + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setTableName("feature_store") + .setKeyspace("test") + .build()) + .build(); + } + + @Override + public Map getFeatureSetSpecs() { + return new HashMap() { + { + put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); + } + }; + } + } + + @BeforeClass + public static void startServer() throws InterruptedException, IOException, TTransportException { + LocalCassandra.start(); + LocalCassandra.createKeyspaceAndTable(); + } + + @Before + public void setUp() { + featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 10, + new HashMap() { + { + put("entity1", Enum.INT64); + put("entity2", Enum.STRING); + } + }, + new HashMap() { + { + put("feature1", Enum.INT64); + put("feature2", Enum.INT64); + } + }); + row = + TestUtil.createFeatureRow( + featureSetSpec, + 100, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("entity2", TestUtil.strValue("a")); + put("feature1", TestUtil.intValue(1)); + put("feature2", TestUtil.intValue(1)); + } + }); + } + + @Rule public transient TestPipeline testPipeline = TestPipeline.create(); + + @AfterClass + public static void cleanUp() { + LocalCassandra.stop(); + } + + @Test + public void testWriteCassandra_happyPath() throws InvalidProtocolBufferException { + PCollection input = testPipeline.apply(Create.of(row)); + + input.apply(new FakeCassandraWriteToStore(featureSetSpec)); + + testPipeline.run(); + + ResultSet resultSet = LocalCassandra.getSession().execute("SELECT * FROM test.feature_store"); + List actualResults = getResults(resultSet); + + List expectedFields = + Arrays.asList( + Field.newBuilder().setName("feature1").setValue(TestUtil.intValue(1)).build(), + Field.newBuilder().setName("feature2").setValue(TestUtil.intValue(1)).build()); + + assertTrue(actualResults.containsAll(expectedFields)); + assertEquals(expectedFields.size(), actualResults.size()); + } + + @Test(timeout = 30000) + public void testWriteCassandra_shouldNotRetrieveExpiredValues() + throws InvalidProtocolBufferException { + // Set max age to 1 second + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 1, + new HashMap() { + { + put("entity1", Enum.INT64); + put("entity2", Enum.STRING); + } + }, + new HashMap() { + { + put("feature1", Enum.INT64); + put("feature2", Enum.INT64); + } + }); + + PCollection input = testPipeline.apply(Create.of(row)); + + input.apply(new FakeCassandraWriteToStore(featureSetSpec)); + + testPipeline.run(); + + while (true) { + ResultSet resultSet = + LocalCassandra.getSession() + .execute("SELECT feature, value, ttl(value) as expiry FROM test.feature_store"); + List results = getResults(resultSet); + if (results.isEmpty()) break; + } + } + + @Test + public void testWriteCassandra_shouldNotOverrideNewerValues() + throws InvalidProtocolBufferException { + FeatureRow olderRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("entity2", TestUtil.strValue("a")); + put("feature1", TestUtil.intValue(3)); + put("feature2", TestUtil.intValue(3)); + } + }); + + PCollection input = testPipeline.apply(Create.of(row, olderRow)); + + input.apply(new FakeCassandraWriteToStore(featureSetSpec)); + + testPipeline.run(); + + ResultSet resultSet = LocalCassandra.getSession().execute("SELECT * FROM test.feature_store"); + List actualResults = getResults(resultSet); + + List expectedFields = + Arrays.asList( + Field.newBuilder().setName("feature1").setValue(TestUtil.intValue(1)).build(), + Field.newBuilder().setName("feature2").setValue(TestUtil.intValue(1)).build()); + + assertTrue(actualResults.containsAll(expectedFields)); + assertEquals(expectedFields.size(), actualResults.size()); + } + + private List getResults(ResultSet resultSet) throws InvalidProtocolBufferException { + List results = new ArrayList<>(); + while (!resultSet.isExhausted()) { + Row row = resultSet.one(); + results.add( + Field.newBuilder() + .setName(row.getString("feature")) + .setValue(Value.parseFrom(row.getBytes("value"))) + .build()); + } + return results; + } +} diff --git a/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java new file mode 100644 index 0000000000..19a0091d17 --- /dev/null +++ b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java @@ -0,0 +1,167 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.util; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.TableMetadata; +import com.datastax.driver.core.schemabuilder.Create; +import com.datastax.driver.core.schemabuilder.SchemaBuilder; +import feast.core.StoreProto.Store.CassandraConfig; +import feast.ingestion.utils.StoreUtil; +import feast.store.serving.cassandra.CassandraMutation; +import feast.test.TestUtil.LocalCassandra; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.thrift.transport.TTransportException; +import org.junit.After; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class CassandraStoreUtilIT { + + @BeforeClass + public static void startServer() throws InterruptedException, IOException, TTransportException { + LocalCassandra.start(); + } + + @After + public void teardown() { + LocalCassandra.stop(); + } + + @Test + public void setupCassandra_shouldCreateKeyspaceAndTable() { + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + } + }) + .build(); + StoreUtil.setupCassandra(config); + + Map actualReplication = + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getReplication(); + Map expectedReplication = + new HashMap() { + { + put("class", "org.apache.cassandra.locator.NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + } + }; + TableMetadata tableMetadata = + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store"); + + Assert.assertEquals(expectedReplication, actualReplication); + Assert.assertNotNull(tableMetadata); + } + + @Test + public void setupCassandra_shouldBeIdempotent_whenTableAlreadyExistsAndSchemaMatches() { + LocalCassandra.createKeyspaceAndTable(); + + // Check table is created + Assert.assertNotNull( + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store")); + + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "SimpleStrategy"); + put("replication_factor", "2"); + } + }) + .build(); + + StoreUtil.setupCassandra(config); + + Assert.assertNotNull( + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store")); + } + + @Test(expected = RuntimeException.class) + public void setupCassandra_shouldThrowException_whenTableNameDoesNotMatchObjectMapper() { + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("test_data_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + } + }) + .build(); + StoreUtil.setupCassandra(config); + } + + @Test(expected = RuntimeException.class) + public void setupCassandra_shouldThrowException_whenTableSchemaDoesNotMatchObjectMapper() { + LocalCassandra.getSession() + .execute( + "CREATE KEYSPACE test " + + "WITH REPLICATION = {" + + "'class': 'SimpleStrategy', 'replication_factor': 2 }"); + + Create createTable = + SchemaBuilder.createTable("test", "feature_store") + .ifNotExists() + .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) + .addClusteringColumn( + "featureName", DataType.text()) // Column name does not match in CassandraMutation + .addStaticColumn(CassandraMutation.VALUE, DataType.blob()); + LocalCassandra.getSession().execute(createTable); + + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "SimpleStrategy"); + put("replication_factor", "2"); + } + }) + .build(); + + StoreUtil.setupCassandra(config); + } +} diff --git a/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java b/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java new file mode 100644 index 0000000000..7ec255e63f --- /dev/null +++ b/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java @@ -0,0 +1,225 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.cassandra; + +import com.google.protobuf.Duration; +import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.test.TestUtil; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.ValueProto.Value; +import feast.types.ValueProto.ValueType.Enum; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.HashMap; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.junit.Rule; +import org.junit.Test; + +public class FeatureRowToCassandraMutationDoFnTest implements Serializable { + + @Rule public transient TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void processElement_shouldCreateCassandraMutation_givenFeatureRow() { + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 10, + new HashMap() { + { + put("entity1", Enum.INT64); + } + }, + new HashMap() { + { + put("feature1", Enum.STRING); + } + }); + FeatureRow featureRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("feature1", TestUtil.strValue("a")); + } + }); + + PCollection input = testPipeline.apply(Create.of(featureRow)); + + PCollection output = + input.apply( + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + new HashMap() { + { + put( + featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), + featureSetSpec); + } + }, + Duration.newBuilder().setSeconds(0).build()))); + + CassandraMutation[] expected = + new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 10) + }; + + PAssert.that(output).containsInAnyOrder(expected); + + testPipeline.run(); + } + + @Test + public void + processElement_shouldCreateCassandraMutations_givenFeatureRowWithMultipleEntitiesAndFeatures() { + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 10, + new HashMap() { + { + put("entity1", Enum.INT64); + put("entity2", Enum.STRING); + } + }, + new HashMap() { + { + put("feature1", Enum.STRING); + put("feature2", Enum.INT64); + } + }); + FeatureRow featureRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("entity2", TestUtil.strValue("b")); + put("feature1", TestUtil.strValue("a")); + put("feature2", TestUtil.intValue(2)); + } + }); + + PCollection input = testPipeline.apply(Create.of(featureRow)); + + PCollection output = + input.apply( + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + new HashMap() { + { + put( + featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), + featureSetSpec); + } + }, + Duration.newBuilder().setSeconds(0).build()))); + + CassandraMutation[] expected = + new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1|entity2=b", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 10), + new CassandraMutation( + "fs:1:entity1=1|entity2=b", + "feature2", + ByteBuffer.wrap(TestUtil.intValue(2).toByteArray()), + 10000000, + 10) + }; + + PAssert.that(output).containsInAnyOrder(expected); + + testPipeline.run(); + } + + @Test + public void processElement_shouldUseDefaultMaxAge_whenMissingMaxAge() { + Duration defaultTtl = Duration.newBuilder().setSeconds(500).build(); + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 0, + new HashMap() { + { + put("entity1", Enum.INT64); + } + }, + new HashMap() { + { + put("feature1", Enum.STRING); + } + }); + FeatureRow featureRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("feature1", TestUtil.strValue("a")); + } + }); + + PCollection input = testPipeline.apply(Create.of(featureRow)); + + PCollection output = + input.apply( + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + new HashMap() { + { + put( + featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), + featureSetSpec); + } + }, + defaultTtl))); + + CassandraMutation[] expected = + new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 500) + }; + + PAssert.that(output).containsInAnyOrder(expected); + + testPipeline.run(); + } +} diff --git a/ingestion/src/test/java/feast/test/TestUtil.java b/ingestion/src/test/java/feast/test/TestUtil.java index ef41f3950a..0fcb9f4e16 100644 --- a/ingestion/src/test/java/feast/test/TestUtil.java +++ b/ingestion/src/test/java/feast/test/TestUtil.java @@ -16,9 +16,14 @@ */ package feast.test; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; import com.google.protobuf.ByteString; +import com.google.protobuf.Timestamp; import com.google.protobuf.util.Timestamps; +import feast.core.FeatureSetProto.EntitySpec; import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.core.FeatureSetProto.FeatureSpec; import feast.ingestion.transform.WriteToStore; import feast.storage.RedisProto.RedisKey; import feast.types.FeatureRowProto.FeatureRow; @@ -33,13 +38,18 @@ import feast.types.ValueProto.StringList; import feast.types.ValueProto.Value; import feast.types.ValueProto.ValueType; +import feast.types.ValueProto.ValueType.Enum; import java.io.IOException; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; import kafka.server.KafkaConfig; import kafka.server.KafkaServerStartable; import org.apache.beam.sdk.PipelineResult; @@ -51,8 +61,11 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.thrift.transport.TTransportException; import org.apache.zookeeper.server.ServerConfig; import org.apache.zookeeper.server.ZooKeeperServerMain; +import org.cassandraunit.dataset.cql.ClassPathCQLDataSet; +import org.cassandraunit.utils.EmbeddedCassandraServerHelper; import org.joda.time.Duration; import redis.embedded.RedisServer; @@ -81,6 +94,39 @@ public static void stop() { } } + public static class LocalCassandra { + + public static void start() throws InterruptedException, IOException, TTransportException { + EmbeddedCassandraServerHelper.startEmbeddedCassandra(); + } + + public static void createKeyspaceAndTable() { + new ClassPathCQLDataSet("embedded-store/LoadCassandra.cql", true, true) + .getCQLStatements() + .forEach(s -> LocalCassandra.getSession().execute(s)); + } + + public static String getHost() { + return EmbeddedCassandraServerHelper.getHost(); + } + + public static int getPort() { + return EmbeddedCassandraServerHelper.getNativeTransportPort(); + } + + public static Cluster getCluster() { + return EmbeddedCassandraServerHelper.getCluster(); + } + + public static Session getSession() { + return EmbeddedCassandraServerHelper.getSession(); + } + + public static void stop() { + EmbeddedCassandraServerHelper.cleanEmbeddedCassandra(); + } + } + public static class LocalKafka { private static KafkaServerStartable server; @@ -163,6 +209,85 @@ public static void publishFeatureRowsToKafka( }); } + /** + * Create a Feature Set Spec. + * + * @param name name of the feature set + * @param version version of the feature set + * @param maxAgeSeconds max age + * @param entities entities provided as map of string to {@link Enum} + * @param features features provided as map of string to {@link Enum} + * @return {@link FeatureSetSpec} + */ + public static FeatureSetSpec createFeatureSetSpec( + String name, + int version, + int maxAgeSeconds, + Map entities, + Map features) { + FeatureSetSpec.Builder featureSetSpec = + FeatureSetSpec.newBuilder() + .setName(name) + .setVersion(version) + .setMaxAge(com.google.protobuf.Duration.newBuilder().setSeconds(maxAgeSeconds).build()); + + for (Entry entity : entities.entrySet()) { + featureSetSpec.addEntities( + EntitySpec.newBuilder().setName(entity.getKey()).setValueType(entity.getValue()).build()); + } + + for (Entry feature : features.entrySet()) { + featureSetSpec.addFeatures( + FeatureSpec.newBuilder() + .setName(feature.getKey()) + .setValueType(feature.getValue()) + .build()); + } + + return featureSetSpec.build(); + } + + /** + * Create a Feature Row. + * + * @param featureSetSpec {@link FeatureSetSpec} + * @param timestampSeconds timestamp given in seconds + * @param fields fields provided as a map name to {@link Value} + * @return {@link FeatureRow} + */ + public static FeatureRow createFeatureRow( + FeatureSetSpec featureSetSpec, long timestampSeconds, Map fields) { + List featureNames = + featureSetSpec.getFeaturesList().stream() + .map(FeatureSpec::getName) + .collect(Collectors.toList()); + List entityNames = + featureSetSpec.getEntitiesList().stream() + .map(EntitySpec::getName) + .collect(Collectors.toList()); + List requiredFields = + Stream.concat(featureNames.stream(), entityNames.stream()).collect(Collectors.toList()); + + if (fields.keySet().containsAll(requiredFields)) { + FeatureRow.Builder featureRow = + FeatureRow.newBuilder() + .setFeatureSet(featureSetSpec.getName() + ":" + featureSetSpec.getVersion()) + .setEventTimestamp(Timestamp.newBuilder().setSeconds(timestampSeconds).build()); + for (Entry field : fields.entrySet()) { + featureRow.addFields( + Field.newBuilder().setName(field.getKey()).setValue(field.getValue()).build()); + } + return featureRow.build(); + } else { + String missingFields = + requiredFields.stream() + .filter(f -> !fields.keySet().contains(f)) + .collect(Collectors.joining(",")); + throw new IllegalArgumentException( + "FeatureRow is missing some fields defined in FeatureSetSpec: " + missingFields); + } + } + /** * Create a Feature Row with random value according to the FeatureSetSpec * @@ -352,15 +477,16 @@ public static Field field(String name, Object value, ValueType.Enum valueType) { /** * This blocking method waits until an ImportJob pipeline has written all elements to the store. - *

- * The pipeline must be in the RUNNING state before calling this method. * - * @param pipelineResult result of running the Pipeline + *

The pipeline must be in the RUNNING state before calling this method. + * + * @param pipelineResult result of running the Pipeline * @param maxWaitDuration wait until this max amount of duration * @throws InterruptedException if the thread is interruped while waiting */ - public static void waitUntilAllElementsAreWrittenToStore(PipelineResult pipelineResult, - Duration maxWaitDuration, Duration checkInterval) throws InterruptedException { + public static void waitUntilAllElementsAreWrittenToStore( + PipelineResult pipelineResult, Duration maxWaitDuration, Duration checkInterval) + throws InterruptedException { if (pipelineResult.getState().isTerminal()) { return; } @@ -409,4 +535,12 @@ public static void waitUntilAllElementsAreWrittenToStore(PipelineResult pipeline } } } + + public static Value intValue(int val) { + return Value.newBuilder().setInt64Val(val).build(); + } + + public static Value strValue(String val) { + return Value.newBuilder().setStringVal(val).build(); + } } diff --git a/ingestion/src/test/resources/embedded-store/LoadCassandra.cql b/ingestion/src/test/resources/embedded-store/LoadCassandra.cql new file mode 100644 index 0000000000..c80da294b7 --- /dev/null +++ b/ingestion/src/test/resources/embedded-store/LoadCassandra.cql @@ -0,0 +1,8 @@ +CREATE KEYSPACE test with replication = {'class':'SimpleStrategy','replication_factor':1}; + +CREATE TABLE test.feature_store( + entities text, + feature text, + value blob, + PRIMARY KEY (entities, feature) +) WITH CLUSTERING ORDER BY (feature DESC); \ No newline at end of file diff --git a/pom.xml b/pom.xml index edf2a0244e..9858674067 100644 --- a/pom.xml +++ b/pom.xml @@ -192,7 +192,7 @@ com.google.guava guava - 26.0-jre + 25.0-jre com.google.protobuf diff --git a/protos/feast/core/Store.proto b/protos/feast/core/Store.proto index e1b8c581a3..1cafce59aa 100644 --- a/protos/feast/core/Store.proto +++ b/protos/feast/core/Store.proto @@ -103,7 +103,13 @@ message Store { // BIGQUERY = 2; - // Unsupported in Feast 0.3 + // Cassandra stores entities as a string partition key, feature as clustering column. + // + // Columns: + // - entities: concatenated string of feature set name and all entities' keys and values + // - feature: clustering column where each feature is a column + // - value: byte array of Value (refer to feast.types.Value) + // CASSANDRA = 3; } @@ -118,8 +124,12 @@ message Store { } message CassandraConfig { - string host = 1; + // - bootstrapHosts: [comma delimited value of hosts] + string bootstrap_hosts = 1; int32 port = 2; + string keyspace = 3; + string table_name = 4; + map replication_options = 5; } message Subscription { From 55356d905b83874a6aad3e0d06c1dd09bcec7a26 Mon Sep 17 00:00:00 2001 From: smadarasmi Date: Mon, 2 Dec 2019 18:37:20 +0700 Subject: [PATCH 2/5] add default expiration to cassandra config for when featureset does not have max age --- .../ingestion/transform/WriteToStore.java | 29 +-- .../FeatureRowToCassandraMutationDoFn.java | 38 +-- ...Test.java => CassandraWriteToStoreIT.java} | 2 +- .../ingestion/util/CassandraStoreUtilIT.java | 131 ++++++++++ .../util/CassandraStoreUtilITTest.java | 167 ------------ ...FeatureRowToCassandraMutationDoFnTest.java | 242 ++++++------------ protos/feast/core/Store.proto | 10 + 7 files changed, 243 insertions(+), 376 deletions(-) rename ingestion/src/test/java/feast/ingestion/transform/{CassandraWriteToStoreITTest.java => CassandraWriteToStoreIT.java} (99%) create mode 100644 ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java delete mode 100644 ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java diff --git a/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java b/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java index c89b95bfb2..eb74fffb1c 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java +++ b/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java @@ -68,8 +68,8 @@ public abstract class WriteToStore extends PTransform, P public static final String METRIC_NAMESPACE = "WriteToStore"; public static final String ELEMENTS_WRITTEN_METRIC = "elements_written"; - private static final Counter elementsWritten = - Metrics.counter(METRIC_NAMESPACE, ELEMENTS_WRITTEN_METRIC); + private static final Counter elementsWritten = Metrics + .counter(METRIC_NAMESPACE, ELEMENTS_WRITTEN_METRIC); public abstract Store getStore(); @@ -155,35 +155,32 @@ public void processElement(ProcessContext context) { break; case CASSANDRA: CassandraConfig cassandraConfig = getStore().getCassandraConfig(); - SerializableFunction mapperFactory = - new CassandraMutationMapperFactory(CassandraMutation.class); + SerializableFunction mapperFactory = new CassandraMutationMapperFactory(CassandraMutation.class); input .apply( "Create CassandraMutation from FeatureRow", - ParDo.of( - new FeatureRowToCassandraMutationDoFn( - getFeatureSetSpecs(), cassandraConfig.getDefaultTtl()))) + ParDo.of(new FeatureRowToCassandraMutationDoFn( + getFeatureSetSpecs(), cassandraConfig.getDefaultTtl())) + ) .apply( CassandraIO.write() .withHosts(Arrays.asList(cassandraConfig.getBootstrapHosts().split(","))) .withPort(cassandraConfig.getPort()) .withKeyspace(cassandraConfig.getKeyspace()) .withEntity(CassandraMutation.class) - .withMapperFactoryFn(mapperFactory)); + .withMapperFactoryFn(mapperFactory) + ); break; default: log.error("Store type '{}' is not supported. No Feature Row will be written.", storeType); break; } - input.apply( - "IncrementWriteToStoreElementsWrittenCounter", - MapElements.into(TypeDescriptors.booleans()) - .via( - (FeatureRow row) -> { - elementsWritten.inc(); - return true; - })); + input.apply("IncrementWriteToStoreElementsWrittenCounter", + MapElements.into(TypeDescriptors.booleans()).via((FeatureRow row) -> { + elementsWritten.inc(); + return true; + })); return PDone.in(input.getPipeline()); } diff --git a/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java b/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java index e416fa9b5a..0292735c58 100644 --- a/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java +++ b/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java @@ -1,19 +1,3 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * Copyright 2018-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 - * - * 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.cassandra; import com.google.protobuf.Duration; @@ -34,8 +18,8 @@ public class FeatureRowToCassandraMutationDoFn extends DoFn { - private static final Logger log = - org.slf4j.LoggerFactory.getLogger(FeatureRowToCassandraMutationDoFn.class); + private static final Logger log = org.slf4j.LoggerFactory + .getLogger(FeatureRowToCassandraMutationDoFn.class); private Map featureSetSpecs; private Map maxAges; @@ -52,16 +36,16 @@ public FeatureRowToCassandraMutationDoFn(Map specs, Dura } } - /** Output a Cassandra mutation object for every feature in the feature row. */ + /** + * Output a Cassandra mutation object for every feature in the feature row. + */ @ProcessElement public void processElement(ProcessContext context) { FeatureRow featureRow = context.element(); try { FeatureSetSpec featureSetSpec = featureSetSpecs.get(featureRow.getFeatureSet()); - Set featureNames = - featureSetSpec.getFeaturesList().stream() - .map(FeatureSpec::getName) - .collect(Collectors.toSet()); + Set featureNames = featureSetSpec.getFeaturesList().stream() + .map(FeatureSpec::getName).collect(Collectors.toSet()); String key = CassandraMutation.keyFromFeatureRow(featureSetSpec, featureRow); Collection mutations = new ArrayList<>(); @@ -73,7 +57,9 @@ public void processElement(ProcessContext context) { field.getName(), ByteBuffer.wrap(field.getValue().toByteArray()), Timestamps.toMicros(featureRow.getEventTimestamp()), - maxAges.get(featureRow.getFeatureSet()))); + maxAges.get(featureRow.getFeatureSet()) + ) + ); } } @@ -81,5 +67,7 @@ public void processElement(ProcessContext context) { } catch (Exception e) { log.error(e.getMessage(), e); } + } -} + +} \ No newline at end of file diff --git a/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java b/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java similarity index 99% rename from ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java rename to ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java index 6a94502b94..4d66be089c 100644 --- a/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreITTest.java +++ b/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java @@ -49,7 +49,7 @@ import org.junit.Rule; import org.junit.Test; -public class CassandraWriteToStoreITTest implements Serializable { +public class CassandraWriteToStoreIT implements Serializable { private FeatureSetSpec featureSetSpec; private FeatureRow row; diff --git a/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java new file mode 100644 index 0000000000..1f1d11d72c --- /dev/null +++ b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java @@ -0,0 +1,131 @@ +package feast.ingestion.util; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.TableMetadata; +import com.datastax.driver.core.schemabuilder.Create; +import com.datastax.driver.core.schemabuilder.SchemaBuilder; +import feast.core.StoreProto.Store.CassandraConfig; +import feast.ingestion.utils.StoreUtil; +import feast.store.serving.cassandra.CassandraMutation; +import feast.test.TestUtil.LocalCassandra; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import org.apache.thrift.transport.TTransportException; +import org.junit.After; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class CassandraStoreUtilIT { + + @BeforeClass + public static void startServer() throws InterruptedException, IOException, TTransportException { + LocalCassandra.start(); + } + + @After + public void teardown() { + LocalCassandra.stop(); + } + + @Test + public void setupCassandra_shouldCreateKeyspaceAndTable() { + CassandraConfig config = CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions(new HashMap() {{ + put("class", "NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + }}) + .build(); + StoreUtil.setupCassandra(config); + + Map actualReplication = LocalCassandra.getCluster() + .getMetadata() + .getKeyspace("test") + .getReplication(); + Map expectedReplication = new HashMap() {{ + put("class", "org.apache.cassandra.locator.NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + }}; + TableMetadata tableMetadata = LocalCassandra.getCluster() + .getMetadata().getKeyspace("test").getTable("feature_store"); + + Assert.assertEquals(expectedReplication, actualReplication); + Assert.assertNotNull(tableMetadata); + } + + @Test + public void setupCassandra_shouldBeIdempotent_whenTableAlreadyExistsAndSchemaMatches() { + LocalCassandra.createKeyspaceAndTable(); + + // Check table is created + Assert.assertNotNull(LocalCassandra.getCluster() + .getMetadata().getKeyspace("test").getTable("feature_store")); + + CassandraConfig config = CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions(new HashMap() {{ + put("class", "SimpleStrategy"); + put("replication_factor", "2"); + }}) + .build(); + + StoreUtil.setupCassandra(config); + + Assert.assertNotNull(LocalCassandra.getCluster() + .getMetadata().getKeyspace("test").getTable("feature_store")); + } + + @Test(expected = RuntimeException.class) + public void setupCassandra_shouldThrowException_whenTableNameDoesNotMatchObjectMapper() { + CassandraConfig config = CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("test_data_store") + .putAllReplicationOptions(new HashMap() {{ + put("class", "NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + }}) + .build(); + StoreUtil.setupCassandra(config); + } + + @Test(expected = RuntimeException.class) + public void setupCassandra_shouldThrowException_whenTableSchemaDoesNotMatchObjectMapper() { + LocalCassandra.getSession().execute("CREATE KEYSPACE test " + + "WITH REPLICATION = {" + + "'class': 'SimpleStrategy', 'replication_factor': 2 }"); + + Create createTable = SchemaBuilder.createTable("test", "feature_store") + .ifNotExists() + .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) + .addClusteringColumn("featureName", DataType.text()) // Column name does not match in CassandraMutation + .addStaticColumn(CassandraMutation.VALUE, DataType.blob()); + LocalCassandra.getSession().execute(createTable); + + CassandraConfig config = CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions(new HashMap() {{ + put("class", "SimpleStrategy"); + put("replication_factor", "2"); + }}) + .build(); + + StoreUtil.setupCassandra(config); + } + +} diff --git a/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java deleted file mode 100644 index 19a0091d17..0000000000 --- a/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilITTest.java +++ /dev/null @@ -1,167 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * Copyright 2018-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 - * - * 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.util; - -import com.datastax.driver.core.DataType; -import com.datastax.driver.core.TableMetadata; -import com.datastax.driver.core.schemabuilder.Create; -import com.datastax.driver.core.schemabuilder.SchemaBuilder; -import feast.core.StoreProto.Store.CassandraConfig; -import feast.ingestion.utils.StoreUtil; -import feast.store.serving.cassandra.CassandraMutation; -import feast.test.TestUtil.LocalCassandra; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.thrift.transport.TTransportException; -import org.junit.After; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -public class CassandraStoreUtilIT { - - @BeforeClass - public static void startServer() throws InterruptedException, IOException, TTransportException { - LocalCassandra.start(); - } - - @After - public void teardown() { - LocalCassandra.stop(); - } - - @Test - public void setupCassandra_shouldCreateKeyspaceAndTable() { - CassandraConfig config = - CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("feature_store") - .putAllReplicationOptions( - new HashMap() { - { - put("class", "NetworkTopologyStrategy"); - put("dc1", "2"); - put("dc2", "3"); - } - }) - .build(); - StoreUtil.setupCassandra(config); - - Map actualReplication = - LocalCassandra.getCluster().getMetadata().getKeyspace("test").getReplication(); - Map expectedReplication = - new HashMap() { - { - put("class", "org.apache.cassandra.locator.NetworkTopologyStrategy"); - put("dc1", "2"); - put("dc2", "3"); - } - }; - TableMetadata tableMetadata = - LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store"); - - Assert.assertEquals(expectedReplication, actualReplication); - Assert.assertNotNull(tableMetadata); - } - - @Test - public void setupCassandra_shouldBeIdempotent_whenTableAlreadyExistsAndSchemaMatches() { - LocalCassandra.createKeyspaceAndTable(); - - // Check table is created - Assert.assertNotNull( - LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store")); - - CassandraConfig config = - CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("feature_store") - .putAllReplicationOptions( - new HashMap() { - { - put("class", "SimpleStrategy"); - put("replication_factor", "2"); - } - }) - .build(); - - StoreUtil.setupCassandra(config); - - Assert.assertNotNull( - LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store")); - } - - @Test(expected = RuntimeException.class) - public void setupCassandra_shouldThrowException_whenTableNameDoesNotMatchObjectMapper() { - CassandraConfig config = - CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("test_data_store") - .putAllReplicationOptions( - new HashMap() { - { - put("class", "NetworkTopologyStrategy"); - put("dc1", "2"); - put("dc2", "3"); - } - }) - .build(); - StoreUtil.setupCassandra(config); - } - - @Test(expected = RuntimeException.class) - public void setupCassandra_shouldThrowException_whenTableSchemaDoesNotMatchObjectMapper() { - LocalCassandra.getSession() - .execute( - "CREATE KEYSPACE test " - + "WITH REPLICATION = {" - + "'class': 'SimpleStrategy', 'replication_factor': 2 }"); - - Create createTable = - SchemaBuilder.createTable("test", "feature_store") - .ifNotExists() - .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) - .addClusteringColumn( - "featureName", DataType.text()) // Column name does not match in CassandraMutation - .addStaticColumn(CassandraMutation.VALUE, DataType.blob()); - LocalCassandra.getSession().execute(createTable); - - CassandraConfig config = - CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("feature_store") - .putAllReplicationOptions( - new HashMap() { - { - put("class", "SimpleStrategy"); - put("replication_factor", "2"); - } - }) - .build(); - - StoreUtil.setupCassandra(config); - } -} diff --git a/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java b/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java index 7ec255e63f..a4680408a5 100644 --- a/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java +++ b/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java @@ -1,19 +1,3 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * Copyright 2018-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 - * - * 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.cassandra; import com.google.protobuf.Duration; @@ -35,60 +19,35 @@ public class FeatureRowToCassandraMutationDoFnTest implements Serializable { - @Rule public transient TestPipeline testPipeline = TestPipeline.create(); + @Rule + public transient TestPipeline testPipeline = TestPipeline.create(); @Test public void processElement_shouldCreateCassandraMutation_givenFeatureRow() { - FeatureSetSpec featureSetSpec = - TestUtil.createFeatureSetSpec( - "fs", - 1, - 10, - new HashMap() { - { - put("entity1", Enum.INT64); - } - }, - new HashMap() { - { - put("feature1", Enum.STRING); - } - }); - FeatureRow featureRow = - TestUtil.createFeatureRow( - featureSetSpec, - 10, - new HashMap() { - { - put("entity1", TestUtil.intValue(1)); - put("feature1", TestUtil.strValue("a")); - } - }); + FeatureSetSpec featureSetSpec = TestUtil.createFeatureSetSpec("fs", 1, 10, + new HashMap() {{ put("entity1", Enum.INT64); }}, + new HashMap() {{ put("feature1", Enum.STRING); }}); + FeatureRow featureRow = TestUtil.createFeatureRow(featureSetSpec, 10, + new HashMap() {{ + put("entity1", TestUtil.intValue(1)); put("feature1", TestUtil.strValue("a")); }}); PCollection input = testPipeline.apply(Create.of(featureRow)); PCollection output = - input.apply( - ParDo.of( - new FeatureRowToCassandraMutationDoFn( - new HashMap() { - { - put( - featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), - featureSetSpec); - } - }, - Duration.newBuilder().setSeconds(0).build()))); - - CassandraMutation[] expected = - new CassandraMutation[] { - new CassandraMutation( - "fs:1:entity1=1", - "feature1", - ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), - 10000000, - 10) - }; + input.apply(ParDo.of(new FeatureRowToCassandraMutationDoFn( + new HashMap() {{ + put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); + }}, Duration.newBuilder().setSeconds(0).build()))); + + CassandraMutation[] expected = new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 10 + ) + }; PAssert.that(output).containsInAnyOrder(expected); @@ -96,68 +55,42 @@ public void processElement_shouldCreateCassandraMutation_givenFeatureRow() { } @Test - public void - processElement_shouldCreateCassandraMutations_givenFeatureRowWithMultipleEntitiesAndFeatures() { - FeatureSetSpec featureSetSpec = - TestUtil.createFeatureSetSpec( - "fs", - 1, - 10, - new HashMap() { - { - put("entity1", Enum.INT64); - put("entity2", Enum.STRING); - } - }, - new HashMap() { - { - put("feature1", Enum.STRING); - put("feature2", Enum.INT64); - } - }); - FeatureRow featureRow = - TestUtil.createFeatureRow( - featureSetSpec, - 10, - new HashMap() { - { - put("entity1", TestUtil.intValue(1)); - put("entity2", TestUtil.strValue("b")); - put("feature1", TestUtil.strValue("a")); - put("feature2", TestUtil.intValue(2)); - } - }); + public void processElement_shouldCreateCassandraMutations_givenFeatureRowWithMultipleEntitiesAndFeatures() { + FeatureSetSpec featureSetSpec = TestUtil.createFeatureSetSpec("fs", 1, 10, + new HashMap() {{ put("entity1", Enum.INT64); put("entity2", Enum.STRING); }}, + new HashMap() {{ put("feature1", Enum.STRING); put("feature2", Enum.INT64); }}); + FeatureRow featureRow = TestUtil.createFeatureRow(featureSetSpec, 10, + new HashMap() {{ + put("entity1", TestUtil.intValue(1)); + put("entity2", TestUtil.strValue("b")); + put("feature1", TestUtil.strValue("a")); + put("feature2", TestUtil.intValue(2)); + }}); PCollection input = testPipeline.apply(Create.of(featureRow)); PCollection output = - input.apply( - ParDo.of( - new FeatureRowToCassandraMutationDoFn( - new HashMap() { - { - put( - featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), - featureSetSpec); - } - }, - Duration.newBuilder().setSeconds(0).build()))); - - CassandraMutation[] expected = - new CassandraMutation[] { - new CassandraMutation( - "fs:1:entity1=1|entity2=b", - "feature1", - ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), - 10000000, - 10), - new CassandraMutation( - "fs:1:entity1=1|entity2=b", - "feature2", - ByteBuffer.wrap(TestUtil.intValue(2).toByteArray()), - 10000000, - 10) - }; + input.apply(ParDo.of(new FeatureRowToCassandraMutationDoFn( + new HashMap() {{ + put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); + }}, Duration.newBuilder().setSeconds(0).build()))); + + CassandraMutation[] expected = new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1|entity2=b", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 10 + ), + new CassandraMutation( + "fs:1:entity1=1|entity2=b", + "feature2", + ByteBuffer.wrap(TestUtil.intValue(2).toByteArray()), + 10000000, + 10 + ) + }; PAssert.that(output).containsInAnyOrder(expected); @@ -167,59 +100,34 @@ public void processElement_shouldCreateCassandraMutation_givenFeatureRow() { @Test public void processElement_shouldUseDefaultMaxAge_whenMissingMaxAge() { Duration defaultTtl = Duration.newBuilder().setSeconds(500).build(); - FeatureSetSpec featureSetSpec = - TestUtil.createFeatureSetSpec( - "fs", - 1, - 0, - new HashMap() { - { - put("entity1", Enum.INT64); - } - }, - new HashMap() { - { - put("feature1", Enum.STRING); - } - }); - FeatureRow featureRow = - TestUtil.createFeatureRow( - featureSetSpec, - 10, - new HashMap() { - { - put("entity1", TestUtil.intValue(1)); - put("feature1", TestUtil.strValue("a")); - } - }); + FeatureSetSpec featureSetSpec = TestUtil.createFeatureSetSpec("fs", 1, 0, + new HashMap() {{ put("entity1", Enum.INT64); }}, + new HashMap() {{ put("feature1", Enum.STRING); }}); + FeatureRow featureRow = TestUtil.createFeatureRow(featureSetSpec, 10, + new HashMap() {{ + put("entity1", TestUtil.intValue(1)); put("feature1", TestUtil.strValue("a")); }}); PCollection input = testPipeline.apply(Create.of(featureRow)); PCollection output = - input.apply( - ParDo.of( - new FeatureRowToCassandraMutationDoFn( - new HashMap() { - { - put( - featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), - featureSetSpec); - } - }, - defaultTtl))); - - CassandraMutation[] expected = - new CassandraMutation[] { - new CassandraMutation( - "fs:1:entity1=1", - "feature1", - ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), - 10000000, - 500) - }; + input.apply(ParDo.of(new FeatureRowToCassandraMutationDoFn( + new HashMap() {{ + put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); + }}, defaultTtl))); + + CassandraMutation[] expected = new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 500 + ) + }; PAssert.that(output).containsInAnyOrder(expected); testPipeline.run(); } + } diff --git a/protos/feast/core/Store.proto b/protos/feast/core/Store.proto index 1cafce59aa..34254cf383 100644 --- a/protos/feast/core/Store.proto +++ b/protos/feast/core/Store.proto @@ -17,6 +17,8 @@ syntax = "proto3"; package feast.core; +import "google/protobuf/duration.proto"; + option java_package = "feast.core"; option java_outer_classname = "StoreProto"; option go_package = "github.com/gojek/feast/sdk/go/protos/feast/core"; @@ -104,12 +106,17 @@ message Store { BIGQUERY = 2; // Cassandra stores entities as a string partition key, feature as clustering column. + // NOTE: This store currently uses max_age defined in FeatureSet for ttl // // Columns: // - entities: concatenated string of feature set name and all entities' keys and values // - feature: clustering column where each feature is a column // - value: byte array of Value (refer to feast.types.Value) // + // Internal columns: + // - writeTime: timestamp of the written record. This is used to ensure that new records are not replaced + // by older ones + // - ttl: expiration time the record. Currently using max_age from feature set spec as ttl CASSANDRA = 3; } @@ -130,6 +137,9 @@ message Store { string keyspace = 3; string table_name = 4; map replication_options = 5; + // Default expiration in seconds to use when FeatureSetSpec does not have max_age defined. + // Specify 0 for no default expiration + google.protobuf.Duration default_ttl = 6; } message Subscription { From 71c97137671059b789adfc863f102e5df5a95e13 Mon Sep 17 00:00:00 2001 From: smadarasmi Date: Wed, 27 Nov 2019 14:21:24 +0700 Subject: [PATCH 3/5] add cassandra serving service * Abstracted OnlineServingService for common implementation of online serving stores * Complete tests remain in RedisServingServiceTest while Cassandra tests only contain basic tests for writes, and some other implementation specific to Cassandra --- CONTRIBUTING.md | 4 +- .../feast/charts/feast-serving/values.yaml | 9 + serving/pom.xml | 21 +- serving/sample_cassandra_config.yml | 13 + .../java/feast/serving/FeastProperties.java | 85 ++++++ .../configuration/ServingServiceConfig.java | 59 +++++ .../service/CassandraServingService.java | 153 +++++++++++ .../serving/service/OnlineServingService.java | 236 +++++++++++++++++ .../serving/service/RedisServingService.java | 212 +++------------ .../java/feast/serving/util/ValueUtil.java | 53 ++++ serving/src/main/resources/application.yml | 20 +- .../CassandraServingServiceITTest.java | 244 ++++++++++++++++++ .../service/CassandraServingServiceTest.java | 117 +++++++++ .../service/RedisServingServiceTest.java | 18 +- .../java/feast/serving/test/TestUtil.java | 81 ++++++ .../embedded-store/LoadCassandra.cql | 8 + 16 files changed, 1133 insertions(+), 200 deletions(-) create mode 100644 serving/sample_cassandra_config.yml create mode 100644 serving/src/main/java/feast/serving/service/CassandraServingService.java create mode 100644 serving/src/main/java/feast/serving/service/OnlineServingService.java create mode 100644 serving/src/main/java/feast/serving/util/ValueUtil.java create mode 100644 serving/src/test/java/feast/serving/service/CassandraServingServiceITTest.java create mode 100644 serving/src/test/java/feast/serving/service/CassandraServingServiceTest.java create mode 100644 serving/src/test/java/feast/serving/test/TestUtil.java create mode 100644 serving/src/test/resources/embedded-store/LoadCassandra.cql diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 06476c0156..e7d6fc6a0e 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -88,8 +88,8 @@ store { #### Starting Feast Serving -Feast Serving requires administrators to provide an **existing** store name in Feast. -An instance of Feast Serving can only retrieve features from a **single** store. +Feast Serving requires administrators to provide an **existing** store name in Feast. +An instance of Feast Serving can only retrieve features from a **single** store. > In order to retrieve features from multiple stores you must start **multiple** instances of Feast serving. If you start multiple Feast serving on a single host, make sure that they are listening on different ports. diff --git a/infra/charts/feast/charts/feast-serving/values.yaml b/infra/charts/feast/charts/feast-serving/values.yaml index b312a40692..c5ccf10869 100644 --- a/infra/charts/feast/charts/feast-serving/values.yaml +++ b/infra/charts/feast/charts/feast-serving/values.yaml @@ -56,6 +56,15 @@ application.yaml: config-path: /etc/feast/feast-serving/store.yaml redis-pool-max-size: 128 redis-pool-max-idle: 64 + cassandra-pool-core-local-connections: 1 + cassandra-pool-max-local-connections: 1 + cassandra-pool-core-remote-connections: 1 + cassandra-pool-max-remote-connections: 1 + cassandra-pool-max-requests-local-connection: 32768 + cassandra-pool-max-requests-remote-connection: 2048 + cassandra-pool-new-local-connection-threshold: 30000 + cassandra-pool-new-remote-connection-threshold: 400 + cassandra-pool-timeout-millis: 0 jobs: staging-location: "" store-type: "" diff --git a/serving/pom.xml b/serving/pom.xml index ab9efaff26..3d41c18b8d 100644 --- a/serving/pom.xml +++ b/serving/pom.xml @@ -141,7 +141,19 @@ redis.clients jedis - + + + com.datastax.cassandra + cassandra-driver-core + 3.4.0 + + + io.netty + * + + + + com.google.guava guava @@ -227,6 +239,13 @@ spring-boot-starter-test test + + + org.cassandraunit + cassandra-unit-shaded + 3.11.2.0 + test + diff --git a/serving/sample_cassandra_config.yml b/serving/sample_cassandra_config.yml new file mode 100644 index 0000000000..ca3d4cbbcc --- /dev/null +++ b/serving/sample_cassandra_config.yml @@ -0,0 +1,13 @@ +name: serving +type: CASSANDRA +cassandra_config: + bootstrap_hosts: localhost + port: 9042 + keyspace: feast + table_name: feature_store + replication_options: + class: SimpleStrategy + replication_factor: 1 +subscriptions: + - name: "*" + version: ">0" diff --git a/serving/src/main/java/feast/serving/FeastProperties.java b/serving/src/main/java/feast/serving/FeastProperties.java index e511835b0a..9a087ca7b5 100644 --- a/serving/src/main/java/feast/serving/FeastProperties.java +++ b/serving/src/main/java/feast/serving/FeastProperties.java @@ -85,6 +85,15 @@ public static class StoreProperties { private String configPath; private int redisPoolMaxSize; private int redisPoolMaxIdle; + private int cassandraPoolCoreLocalConnections; + private int cassandraPoolMaxLocalConnections; + private int cassandraPoolCoreRemoteConnections; + private int cassandraPoolMaxRemoteConnections; + private int cassandraPoolMaxRequestsLocalConnection; + private int cassandraPoolMaxRequestsRemoteConnection; + private int cassandraPoolNewLocalConnectionThreshold; + private int cassandraPoolNewRemoteConnectionThreshold; + private int cassandraPoolTimeoutMillis; public String getConfigPath() { return this.configPath; @@ -98,6 +107,42 @@ public int getRedisPoolMaxIdle() { return this.redisPoolMaxIdle; } + public int getCassandraPoolCoreLocalConnections { + return this.cassandraPoolCoreLocalConnections; + } + + public int getCassandraPoolMaxLocalConnections { + return this.cassandraPoolMaxLocalConnections; + } + + public int getCassandraPoolCoreRemoteConnections { + return this.cassandraPoolCoreRemoteConnections; + } + + public int getCassandraPoolMaxRemoteConnections { + return this.cassandraPoolMaxRemoteConnections; + } + + public int getCassandraPoolMaxRequestsLocalConnection { + return this.cassandraPoolMaxRequestsLocalConnection; + } + + public int getCassandraPoolMaxRequestsRemoteConnection { + return this.cassandraPoolMaxRequestsRemoteConnection; + } + + public int getCassandraPoolNewLocalConnectionThreshold { + return this.cassandraPoolNewLocalConnectionThreshold; + } + + public int getCassandraPoolNewRemoteConnectionThreshold { + return this.cassandraPoolNewRemoteConnectionThreshold; + } + + public int getCassandraPoolTimeoutMillis { + return this.cassandraPoolTimeoutMillis; + } + public void setConfigPath(String configPath) { this.configPath = configPath; } @@ -109,6 +154,46 @@ public void setRedisPoolMaxSize(int redisPoolMaxSize) { public void setRedisPoolMaxIdle(int redisPoolMaxIdle) { this.redisPoolMaxIdle = redisPoolMaxIdle; } + + public void setCassandraPoolCoreLocalConnections(int cassandraPoolCoreLocalConnections) { + this.cassandraPoolCoreLocalConnections = cassandraPoolCoreLocalConnections; + } + + public void setCassandraPoolMaxLocalConnections(int cassandraPoolMaxLocalConnections) { + this.cassandraPoolMaxLocalConnections = cassandraPoolMaxLocalConnections; + } + + public void setCassandraPoolCoreRemoteConnections(int cassandraPoolCoreRemoteConnections) { + this.cassandraPoolCoreRemoteConnections = cassandraPoolCoreRemoteConnections; + } + + public void setCassandraPoolMaxRemoteConnections(int cassandraPoolMaxRemoteConnections) { + this.cassandraPoolMaxRemoteConnections = cassandraPoolMaxRemoteConnections; + } + + public void setCassandraPoolMaxRequestsLocalConnection( + int cassandraPoolMaxRequestsLocalConnection) { + this.cassandraPoolMaxRequestsLocalConnection = cassandraPoolMaxRequestsLocalConnection; + } + + public void setCassandraPoolMaxRequestsRemoteConnection( + int cassandraPoolMaxRequestsRemoteConnection) { + this.cassandraPoolMaxRequestsRemoteConnection = cassandraPoolMaxRequestsRemoteConnection; + } + + public void setCassandraPoolNewLocalConnectionThreshold( + int cassandraPoolNewLocalConnectionThreshold) { + this.cassandraPoolNewLocalConnectionThreshold = cassandraPoolNewLocalConnectionThreshold; + } + + public void setCassandraPoolNewRemoteConnectionThreshold( + int cassandraPoolNewRemoteConnectionThreshold) { + this.cassandraPoolNewRemoteConnectionThreshold = cassandraPoolNewRemoteConnectionThreshold; + } + + public void setCassandraPoolTimeoutMillis(int cassandraPoolTimeoutMillis) { + this.cassandraPoolTimeoutMillis = cassandraPoolTimeoutMillis; + } } public static class JobProperties { diff --git a/serving/src/main/java/feast/serving/configuration/ServingServiceConfig.java b/serving/src/main/java/feast/serving/configuration/ServingServiceConfig.java index 08b9655e3e..81994941d5 100644 --- a/serving/src/main/java/feast/serving/configuration/ServingServiceConfig.java +++ b/serving/src/main/java/feast/serving/configuration/ServingServiceConfig.java @@ -16,6 +16,10 @@ */ package feast.serving.configuration; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.HostDistance; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.Session; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryOptions; import com.google.cloud.storage.Storage; @@ -23,19 +27,26 @@ import feast.core.StoreProto.Store; import feast.core.StoreProto.Store.BigQueryConfig; import feast.core.StoreProto.Store.Builder; +import feast.core.StoreProto.Store.CassandraConfig; import feast.core.StoreProto.Store.RedisConfig; import feast.core.StoreProto.Store.StoreType; import feast.core.StoreProto.Store.Subscription; import feast.serving.FeastProperties; import feast.serving.FeastProperties.JobProperties; +import feast.serving.FeastProperties.StoreProperties; import feast.serving.service.BigQueryServingService; import feast.serving.service.CachedSpecService; +import feast.serving.service.CassandraServingService; import feast.serving.service.JobService; import feast.serving.service.NoopJobService; import feast.serving.service.RedisServingService; import feast.serving.service.ServingService; import io.opentracing.Tracer; +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; import org.slf4j.Logger; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; @@ -76,6 +87,13 @@ private Store setStoreConfig(Store.Builder builder, Map options) .build(); return builder.setBigqueryConfig(bqConfig).build(); case CASSANDRA: + CassandraConfig cassandraConfig = + CassandraConfig.newBuilder() + .setBootstrapHosts(options.get("host")) + .setPort(Integer.parseInt(options.get("port"))) + .setKeyspace(options.get("keyspace")) + .build(); + return builder.setCassandraConfig(cassandraConfig).build(); default: throw new IllegalArgumentException( String.format( @@ -135,6 +153,47 @@ public ServingService servingService( storage); break; case CASSANDRA: + StoreProperties storeProperties = feastProperties.getStore(); + PoolingOptions poolingOptions = new PoolingOptions(); + poolingOptions.setCoreConnectionsPerHost( + HostDistance.LOCAL, storeProperties.getCassandraPoolCoreLocalConnections()); + poolingOptions.setCoreConnectionsPerHost( + HostDistance.REMOTE, storeProperties.getCassandraPoolCoreRemoteConnections()); + poolingOptions.setMaxConnectionsPerHost( + HostDistance.LOCAL, storeProperties.getCassandraPoolMaxLocalConnections()); + poolingOptions.setMaxConnectionsPerHost( + HostDistance.REMOTE, storeProperties.getCassandraPoolMaxRemoteConnections()); + poolingOptions.setMaxRequestsPerConnection( + HostDistance.LOCAL, storeProperties.getCassandraPoolMaxRequestsLocalConnection()); + poolingOptions.setMaxRequestsPerConnection( + HostDistance.REMOTE, storeProperties.getCassandraPoolMaxRequestsRemoteConnection()); + poolingOptions.setNewConnectionThreshold( + HostDistance.LOCAL, storeProperties.getCassandraPoolNewLocalConnectionThreshold()); + poolingOptions.setNewConnectionThreshold( + HostDistance.REMOTE, storeProperties.getCassandraPoolNewRemoteConnectionThreshold()); + poolingOptions.setPoolTimeoutMillis(storeProperties.getCassandraPoolTimeoutMillis()); + CassandraConfig cassandraConfig = store.getCassandraConfig(); + List contactPoints = + Arrays.stream(cassandraConfig.getBootstrapHosts().split(",")) + .map(h -> new InetSocketAddress(h, cassandraConfig.getPort())) + .collect(Collectors.toList()); + Cluster cluster = + Cluster.builder() + .addContactPointsWithPorts(contactPoints) + .withPoolingOptions(poolingOptions) + .build(); + // Session in Cassandra is thread-safe and maintains connections to cluster nodes internally + // Recommended to use one session per keyspace instead of open and close connection for each + // request + Session session = cluster.connect(); + servingService = + new CassandraServingService( + session, + cassandraConfig.getKeyspace(), + cassandraConfig.getTableName(), + specService, + tracer); + break; case UNRECOGNIZED: case INVALID: throw new IllegalArgumentException( diff --git a/serving/src/main/java/feast/serving/service/CassandraServingService.java b/serving/src/main/java/feast/serving/service/CassandraServingService.java new file mode 100644 index 0000000000..e9a7dff8ac --- /dev/null +++ b/serving/src/main/java/feast/serving/service/CassandraServingService.java @@ -0,0 +1,153 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.serving.service; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Timestamp; +import feast.serving.ServingAPIProto.FeatureSetRequest; +import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest.EntityRow; +import feast.serving.util.ValueUtil; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.FieldProto.Field; +import feast.types.ValueProto.Value; +import io.opentracing.Scope; +import io.opentracing.Tracer; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class CassandraServingService extends OnlineServingService { + + private final Session session; + private final String keyspace; + private final String tableName; + private final Tracer tracer; + + public CassandraServingService( + Session session, + String keyspace, + String tableName, + CachedSpecService specService, + Tracer tracer) { + super(specService, tracer); + this.session = session; + this.keyspace = keyspace; + this.tableName = tableName; + this.tracer = tracer; + } + + @Override + List createLookupKeys( + List featureSetEntityNames, + List entityRows, + FeatureSetRequest featureSetRequest) { + try (Scope scope = tracer.buildSpan("Cassandra-makeCassandraKeys").startActive(true)) { + String featureSetId = + String.format("%s:%s", featureSetRequest.getName(), featureSetRequest.getVersion()); + return entityRows.stream() + .map(row -> createCassandraKey(featureSetId, featureSetEntityNames, row)) + .collect(Collectors.toList()); + } + } + + @Override + protected boolean isEmpty(ResultSet response) { + return response.isExhausted(); + } + + /** + * Send a list of get request as an mget + * + * @param keys list of string keys + * @return list of {@link FeatureRow} in primitive byte representation for each key + */ + @Override + protected List getAll(List keys) { + List results = new ArrayList<>(); + for (String key : keys) { + results.add( + session.execute( + QueryBuilder.select() + .column("entities") + .column("feature") + .column("value") + .writeTime("value") + .as("writetime") + .from(keyspace, tableName) + .where(QueryBuilder.eq("entities", key)))); + } + return results; + } + + @Override + FeatureRow parseResponse(ResultSet resultSet) { + List fields = new ArrayList<>(); + Instant instant = Instant.now(); + while (!resultSet.isExhausted()) { + Row row = resultSet.one(); + long microSeconds = row.getLong("writetime"); + instant = + Instant.ofEpochSecond( + TimeUnit.MICROSECONDS.toSeconds(microSeconds), + TimeUnit.MICROSECONDS.toNanos( + Math.floorMod(microSeconds, TimeUnit.SECONDS.toMicros(1)))); + try { + fields.add( + Field.newBuilder() + .setName(row.getString("feature")) + .setValue(Value.parseFrom(ByteBuffer.wrap(row.getBytes("value").array()))) + .build()); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } + } + return FeatureRow.newBuilder() + .addAllFields(fields) + .setEventTimestamp( + Timestamp.newBuilder() + .setSeconds(instant.getEpochSecond()) + .setNanos(instant.getNano()) + .build()) + .build(); + } + + /** + * Create cassandra keys + * + * @param featureSet featureSet reference of the feature. E.g. feature_set_1:1 + * @param featureSetEntityNames entity names that belong to the featureSet + * @param entityRow entityRow to build the key from + * @return String + */ + private static String createCassandraKey( + String featureSet, List featureSetEntityNames, EntityRow entityRow) { + Map fieldsMap = entityRow.getFieldsMap(); + List res = new ArrayList<>(); + for (String entityName : featureSetEntityNames) { + res.add(entityName + "=" + ValueUtil.toString(fieldsMap.get(entityName))); + } + return featureSet + ":" + String.join("|", res); + } +} diff --git a/serving/src/main/java/feast/serving/service/OnlineServingService.java b/serving/src/main/java/feast/serving/service/OnlineServingService.java new file mode 100644 index 0000000000..699d48c121 --- /dev/null +++ b/serving/src/main/java/feast/serving/service/OnlineServingService.java @@ -0,0 +1,236 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.serving.service; + +import static feast.serving.util.Metrics.missingKeyCount; +import static feast.serving.util.Metrics.requestCount; +import static feast.serving.util.Metrics.requestLatency; +import static feast.serving.util.Metrics.staleKeyCount; + +import com.google.common.collect.Maps; +import com.google.protobuf.Duration; +import com.google.protobuf.InvalidProtocolBufferException; +import feast.core.FeatureSetProto.EntitySpec; +import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.serving.ServingAPIProto.FeastServingType; +import feast.serving.ServingAPIProto.FeatureSetRequest; +import feast.serving.ServingAPIProto.GetBatchFeaturesRequest; +import feast.serving.ServingAPIProto.GetBatchFeaturesResponse; +import feast.serving.ServingAPIProto.GetFeastServingInfoRequest; +import feast.serving.ServingAPIProto.GetFeastServingInfoResponse; +import feast.serving.ServingAPIProto.GetJobRequest; +import feast.serving.ServingAPIProto.GetJobResponse; +import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest; +import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest.EntityRow; +import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse; +import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse.FieldValues; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.ValueProto.Value; +import io.grpc.Status; +import io.opentracing.Scope; +import io.opentracing.Tracer; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +abstract class OnlineServingService implements ServingService { + + private final CachedSpecService specService; + private final Tracer tracer; + + OnlineServingService(CachedSpecService specService, Tracer tracer) { + this.specService = specService; + this.tracer = tracer; + } + + @Override + public GetFeastServingInfoResponse getFeastServingInfo( + GetFeastServingInfoRequest getFeastServingInfoRequest) { + return GetFeastServingInfoResponse.newBuilder() + .setType(FeastServingType.FEAST_SERVING_TYPE_ONLINE) + .build(); + } + + @Override + public GetBatchFeaturesResponse getBatchFeatures(GetBatchFeaturesRequest getFeaturesRequest) { + throw Status.UNIMPLEMENTED.withDescription("Method not implemented").asRuntimeException(); + } + + @Override + public GetJobResponse getJob(GetJobRequest getJobRequest) { + throw Status.UNIMPLEMENTED.withDescription("Method not implemented").asRuntimeException(); + } + + /** {@inheritDoc} */ + @Override + public GetOnlineFeaturesResponse getOnlineFeatures(GetOnlineFeaturesRequest request) { + try (Scope scope = + tracer.buildSpan("OnlineServingService-getOnlineFeatures").startActive(true)) { + long startTime = System.currentTimeMillis(); + GetOnlineFeaturesResponse.Builder getOnlineFeaturesResponseBuilder = + GetOnlineFeaturesResponse.newBuilder(); + + List entityRows = request.getEntityRowsList(); + Map> featureValuesMap = + entityRows.stream() + .collect(Collectors.toMap(er -> er, er -> Maps.newHashMap(er.getFieldsMap()))); + + List featureSetRequests = request.getFeatureSetsList(); + for (FeatureSetRequest featureSetRequest : featureSetRequests) { + + FeatureSetSpec featureSetSpec = + specService.getFeatureSet(featureSetRequest.getName(), featureSetRequest.getVersion()); + + List featureSetEntityNames = + featureSetSpec.getEntitiesList().stream() + .map(EntitySpec::getName) + .collect(Collectors.toList()); + + Duration defaultMaxAge = featureSetSpec.getMaxAge(); + if (featureSetRequest.getMaxAge().equals(Duration.getDefaultInstance())) { + featureSetRequest = featureSetRequest.toBuilder().setMaxAge(defaultMaxAge).build(); + } + + sendAndProcessMultiGet( + createLookupKeys(featureSetEntityNames, entityRows, featureSetRequest), + entityRows, + featureValuesMap, + featureSetRequest); + } + List fieldValues = + featureValuesMap.values().stream() + .map(m -> FieldValues.newBuilder().putAllFields(m).build()) + .collect(Collectors.toList()); + requestLatency.labels("getOnlineFeatures").observe(System.currentTimeMillis() - startTime); + return getOnlineFeaturesResponseBuilder.addAllFieldValues(fieldValues).build(); + } + } + + /** + * Create lookup keys for corresponding data stores + * + * @param featureSetEntityNames list of entity names + * @param entityRows list of {@link EntityRow} + * @param featureSetRequest {@link FeatureSetRequest} + * @return list of {@link LookupKeyType} + */ + abstract List createLookupKeys( + List featureSetEntityNames, + List entityRows, + FeatureSetRequest featureSetRequest); + + /** + * Checks whether the response is empty, i.e. feature does not exist in the store + * + * @param response {@link ResponseType} + * @return boolean + */ + protected abstract boolean isEmpty(ResponseType response); + + /** + * Send a list of get requests + * + * @param keys list of {@link LookupKeyType} + * @return list of {@link ResponseType} + */ + protected abstract List getAll(List keys); + + /** + * Parse response from data store to FeatureRow + * + * @param response {@link ResponseType} + * @return {@link FeatureRow} + */ + abstract FeatureRow parseResponse(ResponseType response) throws InvalidProtocolBufferException; + + private List getResponses(List keys) { + try (Scope scope = tracer.buildSpan("OnlineServingService-sendMultiGet").startActive(true)) { + long startTime = System.currentTimeMillis(); + try { + return getAll(keys); + } catch (Exception e) { + throw Status.NOT_FOUND + .withDescription("Unable to retrieve feature from online store") + .withCause(e) + .asRuntimeException(); + } finally { + requestLatency.labels("sendMultiGet").observe(System.currentTimeMillis() - startTime); + } + } + } + + private void sendAndProcessMultiGet( + List keys, + List entityRows, + Map> featureValuesMap, + FeatureSetRequest featureSetRequest) { + List responses = getResponses(keys); + + long startTime = System.currentTimeMillis(); + try (Scope scope = tracer.buildSpan("OnlineServingService-processResponse").startActive(true)) { + String featureSetId = + String.format("%s:%d", featureSetRequest.getName(), featureSetRequest.getVersion()); + Map nullValues = + featureSetRequest.getFeatureNamesList().stream() + .collect( + Collectors.toMap( + name -> featureSetId + ":" + name, name -> Value.newBuilder().build())); + for (int i = 0; i < responses.size(); i++) { + EntityRow entityRow = entityRows.get(i); + Map featureValues = featureValuesMap.get(entityRow); + try { + ResponseType response = responses.get(i); + if (isEmpty(response)) { + missingKeyCount.labels(featureSetRequest.getName()).inc(); + featureValues.putAll(nullValues); + continue; + } + + FeatureRow featureRow = parseResponse(response); + boolean stale = isStale(featureSetRequest, entityRow, featureRow); + if (stale) { + staleKeyCount.labels(featureSetRequest.getName()).inc(); + featureValues.putAll(nullValues); + continue; + } + + requestCount.labels(featureSetRequest.getName()).inc(); + featureRow.getFieldsList().stream() + .filter(f -> featureSetRequest.getFeatureNamesList().contains(f.getName())) + .forEach(f -> featureValues.put(featureSetId + ":" + f.getName(), f.getValue())); + } catch (InvalidProtocolBufferException e) { + e.printStackTrace(); + } + } + } finally { + requestLatency.labels("processResponse").observe(System.currentTimeMillis() - startTime); + } + } + + private static boolean isStale( + FeatureSetRequest featureSetRequest, EntityRow entityRow, FeatureRow featureRow) { + if (featureSetRequest.getMaxAge().equals(Duration.getDefaultInstance())) { + return false; + } + long givenTimestamp = entityRow.getEntityTimestamp().getSeconds(); + if (givenTimestamp == 0) { + givenTimestamp = System.currentTimeMillis() / 1000; + } + long timeDifference = givenTimestamp - featureRow.getEventTimestamp().getSeconds(); + return timeDifference > featureSetRequest.getMaxAge().getSeconds(); + } +} diff --git a/serving/src/main/java/feast/serving/service/RedisServingService.java b/serving/src/main/java/feast/serving/service/RedisServingService.java index 9eaeb17dea..df97c25820 100644 --- a/serving/src/main/java/feast/serving/service/RedisServingService.java +++ b/serving/src/main/java/feast/serving/service/RedisServingService.java @@ -16,29 +16,10 @@ */ package feast.serving.service; -import static feast.serving.util.Metrics.missingKeyCount; -import static feast.serving.util.Metrics.requestCount; -import static feast.serving.util.Metrics.requestLatency; -import static feast.serving.util.Metrics.staleKeyCount; - -import com.google.common.collect.Maps; import com.google.protobuf.AbstractMessageLite; -import com.google.protobuf.Duration; import com.google.protobuf.InvalidProtocolBufferException; -import feast.core.FeatureSetProto.EntitySpec; -import feast.core.FeatureSetProto.FeatureSetSpec; -import feast.serving.ServingAPIProto.FeastServingType; import feast.serving.ServingAPIProto.FeatureSetRequest; -import feast.serving.ServingAPIProto.GetBatchFeaturesRequest; -import feast.serving.ServingAPIProto.GetBatchFeaturesResponse; -import feast.serving.ServingAPIProto.GetFeastServingInfoRequest; -import feast.serving.ServingAPIProto.GetFeastServingInfoResponse; -import feast.serving.ServingAPIProto.GetJobRequest; -import feast.serving.ServingAPIProto.GetJobResponse; -import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest; import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest.EntityRow; -import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse; -import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse.FieldValues; import feast.storage.RedisProto.RedisKey; import feast.types.FeatureRowProto.FeatureRow; import feast.types.FieldProto.Field; @@ -53,88 +34,18 @@ import redis.clients.jedis.Jedis; import redis.clients.jedis.JedisPool; -public class RedisServingService implements ServingService { +public class RedisServingService extends OnlineServingService { private static final Logger log = org.slf4j.LoggerFactory.getLogger(RedisServingService.class); private final JedisPool jedisPool; - private final CachedSpecService specService; private final Tracer tracer; public RedisServingService(JedisPool jedisPool, CachedSpecService specService, Tracer tracer) { + super(specService, tracer); this.jedisPool = jedisPool; - this.specService = specService; this.tracer = tracer; } - /** {@inheritDoc} */ - @Override - public GetFeastServingInfoResponse getFeastServingInfo( - GetFeastServingInfoRequest getFeastServingInfoRequest) { - return GetFeastServingInfoResponse.newBuilder() - .setType(FeastServingType.FEAST_SERVING_TYPE_ONLINE) - .build(); - } - - /** {@inheritDoc} */ - @Override - public GetOnlineFeaturesResponse getOnlineFeatures(GetOnlineFeaturesRequest request) { - try (Scope scope = tracer.buildSpan("Redis-getOnlineFeatures").startActive(true)) { - long startTime = System.currentTimeMillis(); - GetOnlineFeaturesResponse.Builder getOnlineFeaturesResponseBuilder = - GetOnlineFeaturesResponse.newBuilder(); - - List entityRows = request.getEntityRowsList(); - Map> featureValuesMap = - entityRows.stream() - .collect(Collectors.toMap(er -> er, er -> Maps.newHashMap(er.getFieldsMap()))); - - List featureSetRequests = request.getFeatureSetsList(); - for (FeatureSetRequest featureSetRequest : featureSetRequests) { - - FeatureSetSpec featureSetSpec = - specService.getFeatureSet(featureSetRequest.getName(), featureSetRequest.getVersion()); - - List featureSetEntityNames = - featureSetSpec.getEntitiesList().stream() - .map(EntitySpec::getName) - .collect(Collectors.toList()); - - Duration defaultMaxAge = featureSetSpec.getMaxAge(); - if (featureSetRequest.getMaxAge().equals(Duration.getDefaultInstance())) { - featureSetRequest = featureSetRequest.toBuilder().setMaxAge(defaultMaxAge).build(); - } - - List redisKeys = - getRedisKeys(featureSetEntityNames, entityRows, featureSetRequest); - - try { - sendAndProcessMultiGet(redisKeys, entityRows, featureValuesMap, featureSetRequest); - } catch (InvalidProtocolBufferException e) { - throw Status.INTERNAL - .withDescription("Unable to parse protobuf while retrieving feature") - .withCause(e) - .asRuntimeException(); - } - } - List fieldValues = - featureValuesMap.values().stream() - .map(m -> FieldValues.newBuilder().putAllFields(m).build()) - .collect(Collectors.toList()); - requestLatency.labels("getOnlineFeatures").observe(System.currentTimeMillis() - startTime); - return getOnlineFeaturesResponseBuilder.addAllFieldValues(fieldValues).build(); - } - } - - @Override - public GetBatchFeaturesResponse getBatchFeatures(GetBatchFeaturesRequest getFeaturesRequest) { - throw Status.UNIMPLEMENTED.withDescription("Method not implemented").asRuntimeException(); - } - - @Override - public GetJobResponse getJob(GetJobRequest getJobRequest) { - throw Status.UNIMPLEMENTED.withDescription("Method not implemented").asRuntimeException(); - } - /** * Build the redis keys for retrieval from the store. * @@ -143,7 +54,8 @@ public GetJobResponse getJob(GetJobRequest getJobRequest) { * @param featureSetRequest details of the requested featureSet * @return list of RedisKeys */ - private List getRedisKeys( + @Override + List createLookupKeys( List featureSetEntityNames, List entityRows, FeatureSetRequest featureSetRequest) { @@ -158,6 +70,33 @@ private List getRedisKeys( } } + @Override + protected boolean isEmpty(byte[] response) { + return response == null; + } + + /** + * Send a list of get request as an mget + * + * @param keys list of {@link RedisKey} + * @return list of {@link FeatureRow} in primitive byte representation for each {@link RedisKey} + */ + @Override + protected List getAll(List keys) { + Jedis jedis = jedisPool.getResource(); + byte[][] binaryKeys = + keys.stream() + .map(AbstractMessageLite::toByteArray) + .collect(Collectors.toList()) + .toArray(new byte[0][0]); + return jedis.mget(binaryKeys); + } + + @Override + FeatureRow parseResponse(byte[] response) throws InvalidProtocolBufferException { + return FeatureRow.parseFrom(response); + } + /** * Create {@link RedisKey} * @@ -188,93 +127,4 @@ private RedisKey makeRedisKey( } return builder.build(); } - - private void sendAndProcessMultiGet( - List redisKeys, - List entityRows, - Map> featureValuesMap, - FeatureSetRequest featureSetRequest) - throws InvalidProtocolBufferException { - - List jedisResps = sendMultiGet(redisKeys); - long startTime = System.currentTimeMillis(); - try (Scope scope = tracer.buildSpan("Redis-processResponse").startActive(true)) { - String featureSetId = - String.format("%s:%d", featureSetRequest.getName(), featureSetRequest.getVersion()); - - Map nullValues = - featureSetRequest.getFeatureNamesList().stream() - .collect( - Collectors.toMap( - name -> featureSetId + ":" + name, name -> Value.newBuilder().build())); - - for (int i = 0; i < jedisResps.size(); i++) { - EntityRow entityRow = entityRows.get(i); - Map featureValues = featureValuesMap.get(entityRow); - - byte[] jedisResponse = jedisResps.get(i); - if (jedisResponse == null) { - missingKeyCount.labels(featureSetRequest.getName()).inc(); - featureValues.putAll(nullValues); - continue; - } - - FeatureRow featureRow = FeatureRow.parseFrom(jedisResponse); - - boolean stale = isStale(featureSetRequest, entityRow, featureRow); - if (stale) { - staleKeyCount.labels(featureSetRequest.getName()).inc(); - featureValues.putAll(nullValues); - continue; - } - - requestCount.labels(featureSetRequest.getName()).inc(); - featureRow.getFieldsList().stream() - .filter(f -> featureSetRequest.getFeatureNamesList().contains(f.getName())) - .forEach(f -> featureValues.put(featureSetId + ":" + f.getName(), f.getValue())); - } - } finally { - requestLatency.labels("processResponse").observe(System.currentTimeMillis() - startTime); - } - } - - private boolean isStale( - FeatureSetRequest featureSetRequest, EntityRow entityRow, FeatureRow featureRow) { - if (featureSetRequest.getMaxAge().equals(Duration.getDefaultInstance())) { - return false; - } - long givenTimestamp = entityRow.getEntityTimestamp().getSeconds(); - if (givenTimestamp == 0) { - givenTimestamp = System.currentTimeMillis() / 1000; - } - long timeDifference = givenTimestamp - featureRow.getEventTimestamp().getSeconds(); - return timeDifference > featureSetRequest.getMaxAge().getSeconds(); - } - - /** - * Send a list of get request as an mget - * - * @param keys list of {@link RedisKey} - * @return list of {@link FeatureRow} in primitive byte representation for each {@link RedisKey} - */ - private List sendMultiGet(List keys) { - try (Scope scope = tracer.buildSpan("Redis-sendMultiGet").startActive(true)) { - long startTime = System.currentTimeMillis(); - try (Jedis jedis = jedisPool.getResource()) { - byte[][] binaryKeys = - keys.stream() - .map(AbstractMessageLite::toByteArray) - .collect(Collectors.toList()) - .toArray(new byte[0][0]); - return jedis.mget(binaryKeys); - } catch (Exception e) { - throw Status.NOT_FOUND - .withDescription("Unable to retrieve feature from Redis") - .withCause(e) - .asRuntimeException(); - } finally { - requestLatency.labels("sendMultiGet").observe(System.currentTimeMillis() - startTime); - } - } - } } diff --git a/serving/src/main/java/feast/serving/util/ValueUtil.java b/serving/src/main/java/feast/serving/util/ValueUtil.java new file mode 100644 index 0000000000..e3ede6af98 --- /dev/null +++ b/serving/src/main/java/feast/serving/util/ValueUtil.java @@ -0,0 +1,53 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.serving.util; + +import feast.types.ValueProto.Value; + +public class ValueUtil { + + public static String toString(Value value) { + String strValue; + switch (value.getValCase()) { + case BYTES_VAL: + strValue = value.getBytesVal().toString(); + break; + case STRING_VAL: + strValue = value.getStringVal(); + break; + case INT32_VAL: + strValue = String.valueOf(value.getInt32Val()); + break; + case INT64_VAL: + strValue = String.valueOf(value.getInt64Val()); + break; + case DOUBLE_VAL: + strValue = String.valueOf(value.getDoubleVal()); + break; + case FLOAT_VAL: + strValue = String.valueOf(value.getFloatVal()); + break; + case BOOL_VAL: + strValue = String.valueOf(value.getBoolVal()); + break; + default: + throw new IllegalArgumentException( + String.format("toString method not supported for type %s", value.getValCase())); + } + return strValue; + } +} diff --git a/serving/src/main/resources/application.yml b/serving/src/main/resources/application.yml index 2daa83fbfb..e18f49207b 100644 --- a/serving/src/main/resources/application.yml +++ b/serving/src/main/resources/application.yml @@ -1,7 +1,7 @@ feast: # This value is retrieved from project.version properties in pom.xml # https://docs.spring.io/spring-boot/docs/current/reference/html/ - version: @project.version@ +# version: @project.version@ # GRPC service address for Feast Core # Feast Serving requires connection to Feast Core to retrieve and reload Feast metadata (e.g. FeatureSpecs, Store information) core-host: ${FEAST_CORE_HOST:localhost} @@ -24,6 +24,24 @@ feast: redis-pool-max-size: ${FEAST_REDIS_POOL_MAX_SIZE:128} # If serving redis, the redis pool max idle conns redis-pool-max-idle: ${FEAST_REDIS_POOL_MAX_IDLE:16} + # If serving cassandra, minimum connection for local host (one in same data center) + cassandra-pool-core-local-connections: ${FEAST_CASSANDRA_CORE_LOCAL_CONNECTIONS:1} + # If serving cassandra, maximum connection for local host (one in same data center) + cassandra-pool-max-local-connections: ${FEAST_CASSANDRA_MAX_LOCAL_CONNECTIONS:1} + # If serving cassandra, minimum connection for remote host (one in remote data center) + cassandra-pool-core-remote-connections: ${FEAST_CASSANDRA_CORE_REMOTE_CONNECTIONS:1} + # If serving cassandra, maximum connection for remote host (one in same data center) + cassandra-pool-max-remote-connections: ${FEAST_CASSANDRA_MAX_REMOTE_CONNECTIONS:1} + # If serving cassandra, maximum number of concurrent requests per local connection (one in same data center) + cassandra-pool-max-requests-local-connection: ${FEAST_CASSANDRA_MAX_REQUESTS_LOCAL_CONNECTION:32768} + # If serving cassandra, maximum number of concurrent requests per remote connection (one in remote data center) + cassandra-pool-max-requests-remote-connection: ${FEAST_CASSANDRA_MAX_REQUESTS_REMOTE_CONNECTION:2048} + # If serving cassandra, number of requests which trigger opening of new local connection (if it is available) + cassandra-pool-new-local-connection-threshold: ${FEAST_CASSANDRA_NEW_LOCAL_CONNECTION_THRESHOLD:30000} + # If serving cassandra, number of requests which trigger opening of new remote connection (if it is available) + cassandra-pool-new-remote-connection-threshold: ${FEAST_CASSANDRA_NEW_REMOTE_CONNECTION_THRESHOLD:400} + # If serving cassandra, number of milliseconds to wait to acquire connection (after that go to next available host in query plan) + cassandra-pool-timeout-millis: ${FEAST_CASSANDRA_POOL_TIMEOUT_MILLIS:0} jobs: # job-staging-location specifies the URI to store intermediate files for batch serving. diff --git a/serving/src/test/java/feast/serving/service/CassandraServingServiceITTest.java b/serving/src/test/java/feast/serving/service/CassandraServingServiceITTest.java new file mode 100644 index 0000000000..a1778251a3 --- /dev/null +++ b/serving/src/test/java/feast/serving/service/CassandraServingServiceITTest.java @@ -0,0 +1,244 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.serving.service; + +import static feast.serving.test.TestUtil.intValue; +import static feast.serving.test.TestUtil.responseToMapList; +import static feast.serving.test.TestUtil.strValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.querybuilder.Insert; +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.datastax.driver.core.utils.Bytes; +import com.google.common.collect.Lists; +import com.google.protobuf.InvalidProtocolBufferException; +import com.google.protobuf.Timestamp; +import feast.core.FeatureSetProto.EntitySpec; +import feast.core.FeatureSetProto.FeatureSetSpec; +import feast.serving.ServingAPIProto.FeatureSetRequest; +import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest; +import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest.EntityRow; +import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse; +import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse.FieldValues; +import feast.serving.test.TestUtil.LocalCassandra; +import feast.types.FeatureRowProto.FeatureRow; +import feast.types.ValueProto.Value; +import io.opentracing.Tracer; +import io.opentracing.Tracer.SpanBuilder; +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.thrift.transport.TTransportException; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.Mockito; + +public class CassandraServingServiceITTest { + + @Mock CachedSpecService specService; + + @Mock Tracer tracer; + + private CassandraServingService cassandraServingService; + private Session session; + + @BeforeClass + public static void startServer() throws InterruptedException, IOException, TTransportException { + LocalCassandra.start(); + LocalCassandra.createKeyspaceAndTable(); + } + + @Before + public void setup() { + initMocks(this); + FeatureSetSpec featureSetSpec = + FeatureSetSpec.newBuilder() + .addEntities(EntitySpec.newBuilder().setName("entity1")) + .addEntities(EntitySpec.newBuilder().setName("entity2")) + .build(); + + when(specService.getFeatureSet("featureSet", 1)).thenReturn(featureSetSpec); + when(tracer.buildSpan(ArgumentMatchers.any())).thenReturn(Mockito.mock(SpanBuilder.class)); + + session = + new Cluster.Builder() + .addContactPoints(LocalCassandra.getHost()) + .withPort(LocalCassandra.getPort()) + .build() + .connect(); + + populateTable(session); + + cassandraServingService = + new CassandraServingService(session, "test", "feature_store", specService, tracer); + } + + private void populateTable(Session session) { + session.execute( + insertQuery( + "test", "feature_store", "featureSet:1:entity1=1|entity2=a", "feature1", intValue(1))); + session.execute( + insertQuery( + "test", "feature_store", "featureSet:1:entity1=1|entity2=a", "feature2", intValue(1))); + session.execute( + insertQuery( + "test", "feature_store", "featureSet:1:entity1=2|entity2=b", "feature1", intValue(1))); + session.execute( + insertQuery( + "test", "feature_store", "featureSet:1:entity1=2|entity2=b", "feature2", intValue(1))); + } + + @AfterClass + public static void cleanUp() { + LocalCassandra.stop(); + } + + @Test + public void shouldReturnResponseWithValuesIfKeysPresent() { + GetOnlineFeaturesRequest request = + GetOnlineFeaturesRequest.newBuilder() + .addFeatureSets( + FeatureSetRequest.newBuilder() + .setName("featureSet") + .setVersion(1) + .addAllFeatureNames(Lists.newArrayList("feature1", "feature2")) + .build()) + .addEntityRows( + EntityRow.newBuilder() + .setEntityTimestamp(Timestamp.newBuilder().setSeconds(100)) + .putFields("entity1", intValue(1)) + .putFields("entity2", strValue("a"))) + .addEntityRows( + EntityRow.newBuilder() + .setEntityTimestamp(Timestamp.newBuilder().setSeconds(100)) + .putFields("entity1", intValue(2)) + .putFields("entity2", strValue("b"))) + .build(); + + GetOnlineFeaturesResponse expected = + GetOnlineFeaturesResponse.newBuilder() + .addFieldValues( + FieldValues.newBuilder() + .putFields("entity1", intValue(1)) + .putFields("entity2", strValue("a")) + .putFields("featureSet:1:feature1", intValue(1)) + .putFields("featureSet:1:feature2", intValue(1))) + .addFieldValues( + FieldValues.newBuilder() + .putFields("entity1", intValue(2)) + .putFields("entity2", strValue("b")) + .putFields("featureSet:1:feature1", intValue(1)) + .putFields("featureSet:1:feature2", intValue(1))) + .build(); + GetOnlineFeaturesResponse actual = cassandraServingService.getOnlineFeatures(request); + + assertThat( + responseToMapList(actual), containsInAnyOrder(responseToMapList(expected).toArray())); + } + + @Test + public void shouldReturnResponseWithUnsetValuesIfKeysNotPresent() { + GetOnlineFeaturesRequest request = + GetOnlineFeaturesRequest.newBuilder() + .addFeatureSets( + FeatureSetRequest.newBuilder() + .setName("featureSet") + .setVersion(1) + .addAllFeatureNames(Lists.newArrayList("feature1", "feature2")) + .build()) + .addEntityRows( + EntityRow.newBuilder() + .setEntityTimestamp(Timestamp.newBuilder().setSeconds(100)) + .putFields("entity1", intValue(1)) + .putFields("entity2", strValue("a"))) + // Non-existing entity keys + .addEntityRows( + EntityRow.newBuilder() + .setEntityTimestamp(Timestamp.newBuilder().setSeconds(100)) + .putFields("entity1", intValue(55)) + .putFields("entity2", strValue("ff"))) + .build(); + + GetOnlineFeaturesResponse expected = + GetOnlineFeaturesResponse.newBuilder() + .addFieldValues( + FieldValues.newBuilder() + .putFields("entity1", intValue(1)) + .putFields("entity2", strValue("a")) + .putFields("featureSet:1:feature1", intValue(1)) + .putFields("featureSet:1:feature2", intValue(1))) + // Missing keys will return empty values + .addFieldValues( + FieldValues.newBuilder() + .putFields("entity1", intValue(55)) + .putFields("entity2", strValue("ff")) + .putFields("featureSet:1:feature1", Value.newBuilder().build()) + .putFields("featureSet:1:feature2", Value.newBuilder().build())) + .build(); + GetOnlineFeaturesResponse actual = cassandraServingService.getOnlineFeatures(request); + + assertThat( + responseToMapList(actual), containsInAnyOrder(responseToMapList(expected).toArray())); + } + + // This test should fail if cassandra no longer stores write time as microseconds or if we change + // the way we parse microseconds to com.google.protobuf.Timestamp + @Test + public void shouldInsertAndParseWriteTimestampInMicroSeconds() + throws InvalidProtocolBufferException { + session.execute( + "INSERT INTO test.feature_store (entities, feature, value)\n" + + " VALUES ('ENT1', 'FEAT1'," + + Bytes.toHexString(Value.newBuilder().build().toByteArray()) + + ")\n" + + " USING TIMESTAMP 1574318287123456;"); + + ResultSet resultSet = + session.execute( + QueryBuilder.select() + .column("entities") + .column("feature") + .column("value") + .writeTime("value") + .as("writetime") + .from("test", "feature_store") + .where(QueryBuilder.eq("entities", "ENT1"))); + FeatureRow featureRow = cassandraServingService.parseResponse(resultSet); + + Assert.assertEquals( + Timestamp.newBuilder().setSeconds(1574318287).setNanos(123456000).build(), + featureRow.getEventTimestamp()); + } + + private Insert insertQuery( + String database, String table, String key, String featureName, Value value) { + return QueryBuilder.insertInto(database, table) + .value("entities", key) + .value("feature", featureName) + .value("value", ByteBuffer.wrap(value.toByteArray())); + } +} diff --git a/serving/src/test/java/feast/serving/service/CassandraServingServiceTest.java b/serving/src/test/java/feast/serving/service/CassandraServingServiceTest.java new file mode 100644 index 0000000000..f965b14a64 --- /dev/null +++ b/serving/src/test/java/feast/serving/service/CassandraServingServiceTest.java @@ -0,0 +1,117 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.serving.service; + +import static feast.serving.test.TestUtil.intValue; +import static feast.serving.test.TestUtil.strValue; +import static org.mockito.Mockito.when; +import static org.mockito.MockitoAnnotations.initMocks; + +import com.datastax.driver.core.Session; +import feast.serving.ServingAPIProto.FeatureSetRequest; +import feast.serving.ServingAPIProto.GetOnlineFeaturesRequest.EntityRow; +import io.opentracing.Tracer; +import io.opentracing.Tracer.SpanBuilder; +import java.util.ArrayList; +import java.util.List; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mock; +import org.mockito.Mockito; + +public class CassandraServingServiceTest { + + @Mock Session session; + + @Mock CachedSpecService specService; + + @Mock Tracer tracer; + + private CassandraServingService cassandraServingService; + + @Before + public void setUp() { + initMocks(this); + + when(tracer.buildSpan(ArgumentMatchers.any())).thenReturn(Mockito.mock(SpanBuilder.class)); + + cassandraServingService = + new CassandraServingService(session, "test", "feature_store", specService, tracer); + } + + @Test + public void shouldConstructCassandraKeyCorrectly() { + List cassandraKeys = + cassandraServingService.createLookupKeys( + new ArrayList() { + { + add("entity1"); + add("entity2"); + } + }, + new ArrayList() { + { + add( + EntityRow.newBuilder() + .putFields("entity1", intValue(1)) + .putFields("entity2", strValue("a")) + .build()); + add( + EntityRow.newBuilder() + .putFields("entity1", intValue(2)) + .putFields("entity2", strValue("b")) + .build()); + } + }, + FeatureSetRequest.newBuilder().setName("featureSet").setVersion(1).build()); + + List expectedKeys = + new ArrayList() { + { + add("featureSet:1:entity1=1|entity2=a"); + add("featureSet:1:entity1=2|entity2=b"); + } + }; + + Assert.assertEquals(expectedKeys, cassandraKeys); + } + + @Test(expected = Exception.class) + public void shouldThrowExceptionWhenCannotConstructCassandraKey() { + List cassandraKeys = + cassandraServingService.createLookupKeys( + new ArrayList() { + { + add("entity1"); + add("entity2"); + } + }, + new ArrayList() { + { + add(EntityRow.newBuilder().putFields("entity1", intValue(1)).build()); + add( + EntityRow.newBuilder() + .putFields("entity1", intValue(2)) + .putFields("entity2", strValue("b")) + .build()); + } + }, + FeatureSetRequest.newBuilder().setName("featureSet").setVersion(1).build()); + } +} diff --git a/serving/src/test/java/feast/serving/service/RedisServingServiceTest.java b/serving/src/test/java/feast/serving/service/RedisServingServiceTest.java index 890699db6d..dd448fdf2b 100644 --- a/serving/src/test/java/feast/serving/service/RedisServingServiceTest.java +++ b/serving/src/test/java/feast/serving/service/RedisServingServiceTest.java @@ -16,6 +16,9 @@ */ package feast.serving.service; +import static feast.serving.test.TestUtil.intValue; +import static feast.serving.test.TestUtil.responseToMapList; +import static feast.serving.test.TestUtil.strValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.mockito.Mockito.when; @@ -39,7 +42,6 @@ import io.opentracing.Tracer; import io.opentracing.Tracer.SpanBuilder; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.junit.Before; import org.junit.Test; @@ -525,20 +527,6 @@ public void shouldFilterOutUndesiredRows() { responseToMapList(actual), containsInAnyOrder(responseToMapList(expected).toArray())); } - private List> responseToMapList(GetOnlineFeaturesResponse response) { - return response.getFieldValuesList().stream() - .map(FieldValues::getFieldsMap) - .collect(Collectors.toList()); - } - - private Value intValue(int val) { - return Value.newBuilder().setInt64Val(val).build(); - } - - private Value strValue(String val) { - return Value.newBuilder().setStringVal(val).build(); - } - private FeatureSetSpec getFeatureSetSpec() { return FeatureSetSpec.newBuilder() .addEntities(EntitySpec.newBuilder().setName("entity1")) diff --git a/serving/src/test/java/feast/serving/test/TestUtil.java b/serving/src/test/java/feast/serving/test/TestUtil.java new file mode 100644 index 0000000000..9c53359071 --- /dev/null +++ b/serving/src/test/java/feast/serving/test/TestUtil.java @@ -0,0 +1,81 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * Copyright 2018-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 + * + * 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.serving.test; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; +import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse; +import feast.serving.ServingAPIProto.GetOnlineFeaturesResponse.FieldValues; +import feast.types.ValueProto.Value; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.thrift.transport.TTransportException; +import org.cassandraunit.dataset.cql.ClassPathCQLDataSet; +import org.cassandraunit.utils.EmbeddedCassandraServerHelper; + +@SuppressWarnings("WeakerAccess") +public class TestUtil { + + public static class LocalCassandra { + + public static void start() throws InterruptedException, IOException, TTransportException { + EmbeddedCassandraServerHelper.startEmbeddedCassandra(); + } + + public static void createKeyspaceAndTable() { + new ClassPathCQLDataSet("embedded-store/LoadCassandra.cql", true, true) + .getCQLStatements() + .forEach(s -> LocalCassandra.getSession().execute(s)); + } + + public static String getHost() { + return EmbeddedCassandraServerHelper.getHost(); + } + + public static int getPort() { + return EmbeddedCassandraServerHelper.getNativeTransportPort(); + } + + public static Cluster getCluster() { + return EmbeddedCassandraServerHelper.getCluster(); + } + + public static Session getSession() { + return EmbeddedCassandraServerHelper.getSession(); + } + + public static void stop() { + EmbeddedCassandraServerHelper.cleanEmbeddedCassandra(); + } + } + + public static List> responseToMapList(GetOnlineFeaturesResponse response) { + return response.getFieldValuesList().stream() + .map(FieldValues::getFieldsMap) + .collect(Collectors.toList()); + } + + public static Value intValue(int val) { + return Value.newBuilder().setInt64Val(val).build(); + } + + public static Value strValue(String val) { + return Value.newBuilder().setStringVal(val).build(); + } +} diff --git a/serving/src/test/resources/embedded-store/LoadCassandra.cql b/serving/src/test/resources/embedded-store/LoadCassandra.cql new file mode 100644 index 0000000000..c80da294b7 --- /dev/null +++ b/serving/src/test/resources/embedded-store/LoadCassandra.cql @@ -0,0 +1,8 @@ +CREATE KEYSPACE test with replication = {'class':'SimpleStrategy','replication_factor':1}; + +CREATE TABLE test.feature_store( + entities text, + feature text, + value blob, + PRIMARY KEY (entities, feature) +) WITH CLUSTERING ORDER BY (feature DESC); \ No newline at end of file From bc8f3bc354c3abed139216074ee9a87e6f188a10 Mon Sep 17 00:00:00 2001 From: smadarasmi Date: Wed, 27 Nov 2019 14:31:48 +0700 Subject: [PATCH 4/5] update documentation to reflect current API and add cassandra store to docs --- CONTRIBUTING.md | 95 +++++++++++++------ .../java/feast/serving/FeastProperties.java | 18 ++-- 2 files changed, 76 insertions(+), 37 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e7d6fc6a0e..061031a4e8 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -54,35 +54,11 @@ mvn --projects core spring-boot:run # If Feast Core starts successfully, verify the correct Stores are registered # correctly, for example by using grpc_cli. -grpc_cli call localhost:6565 GetStores '' +grpc_cli call localhost:6565 ListStores '' -# Should return something similar to the following. -# Note that you should change BigQuery projectId and datasetId accordingly -# in "$FEAST_HOME/core/src/main/resources/application.yml" - -store { - name: "SERVING" - type: REDIS - subscriptions { - name: "*" - version: ">0" - } - redis_config { - host: "localhost" - port: 6379 - } -} -store { - name: "WAREHOUSE" - type: BIGQUERY - subscriptions { - name: "*" - version: ">0" - } - bigquery_config { - project_id: "my-google-project-id" - dataset_id: "my-bigquery-dataset-id" - } +# Should return something similar to the following if you have not updated any stores +{ + "store": [] } ``` @@ -105,6 +81,69 @@ grpc_cli call localhost:6566 GetFeastServingType '' type: FEAST_SERVING_TYPE_ONLINE ``` +#### Updating a store + +Create a new Store by sending a request to Feast Core. + +``` +# Example of updating a redis store + +grpc_cli call localhost:6565 UpdateStore ' +store { + name: "SERVING" + type: REDIS + subscriptions { + name: "*" + version: ">0" + } + redis_config { + host: "localhost" + port: 6379 + } +} +' + +# Other supported stores examples (replacing redis_config): +# BigQuery +bigquery_config { + project_id: "my-google-project-id" + dataset_id: "my-bigquery-dataset-id" +} + +# Cassandra: two options in cassandra depending on replication strategy +# See details: https://docs.datastax.com/en/cassandra/3.0/cassandra/architecture/archDataDistributeReplication.html +# +# Please note that table name must be "feature_store" as is specified in the @Table annotation of the +# datastax object mapper + +# SimpleStrategy +cassandra_config { + bootstrap_hosts: "localhost" + port: 9042 + keyspace: "feast" + table_name: "feature_store" + replication_options { + class: "SimpleStrategy" + replication_factor: 1 + } +} + +# NetworkTopologyStrategy +cassandra_config { + bootstrap_hosts: "localhost" + port: 9042 + keyspace: "feast" + table_name: "feature_store" + replication_options { + class: "NetworkTopologyStrategy" + east: 2 + west: 2 + } +} + +# To check that the Stores has been updated correctly. +grpc_cli call localhost:6565 ListStores '' +``` #### Registering a FeatureSet diff --git a/serving/src/main/java/feast/serving/FeastProperties.java b/serving/src/main/java/feast/serving/FeastProperties.java index 9a087ca7b5..356f808127 100644 --- a/serving/src/main/java/feast/serving/FeastProperties.java +++ b/serving/src/main/java/feast/serving/FeastProperties.java @@ -107,39 +107,39 @@ public int getRedisPoolMaxIdle() { return this.redisPoolMaxIdle; } - public int getCassandraPoolCoreLocalConnections { + public int getCassandraPoolCoreLocalConnections() { return this.cassandraPoolCoreLocalConnections; } - public int getCassandraPoolMaxLocalConnections { + public int getCassandraPoolMaxLocalConnections() { return this.cassandraPoolMaxLocalConnections; } - public int getCassandraPoolCoreRemoteConnections { + public int getCassandraPoolCoreRemoteConnections() { return this.cassandraPoolCoreRemoteConnections; } - public int getCassandraPoolMaxRemoteConnections { + public int getCassandraPoolMaxRemoteConnections() { return this.cassandraPoolMaxRemoteConnections; } - public int getCassandraPoolMaxRequestsLocalConnection { + public int getCassandraPoolMaxRequestsLocalConnection() { return this.cassandraPoolMaxRequestsLocalConnection; } - public int getCassandraPoolMaxRequestsRemoteConnection { + public int getCassandraPoolMaxRequestsRemoteConnection() { return this.cassandraPoolMaxRequestsRemoteConnection; } - public int getCassandraPoolNewLocalConnectionThreshold { + public int getCassandraPoolNewLocalConnectionThreshold() { return this.cassandraPoolNewLocalConnectionThreshold; } - public int getCassandraPoolNewRemoteConnectionThreshold { + public int getCassandraPoolNewRemoteConnectionThreshold() { return this.cassandraPoolNewRemoteConnectionThreshold; } - public int getCassandraPoolTimeoutMillis { + public int getCassandraPoolTimeoutMillis() { return this.cassandraPoolTimeoutMillis; } From b8d1025e9833894ca5f3ef722a3a6ddb7e7be600 Mon Sep 17 00:00:00 2001 From: smadarasmi Date: Fri, 3 Jan 2020 17:33:45 +0700 Subject: [PATCH 5/5] docs update, spotless check, and bug fix on cassandra schema --- .../ingestion/transform/WriteToStore.java | 29 ++- .../java/feast/ingestion/utils/StoreUtil.java | 2 +- .../FeatureRowToCassandraMutationDoFn.java | 38 ++- .../transform/CassandraWriteToStoreIT.java | 34 ++- .../ingestion/util/CassandraStoreUtilIT.java | 176 ++++++++----- ...FeatureRowToCassandraMutationDoFnTest.java | 242 ++++++++++++------ .../src/test/java/feast/test/TestUtil.java | 9 +- .../embedded-store/LoadCassandra.cql | 8 - protos/feast/core/Store.proto | 9 + 9 files changed, 350 insertions(+), 197 deletions(-) delete mode 100644 ingestion/src/test/resources/embedded-store/LoadCassandra.cql diff --git a/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java b/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java index eb74fffb1c..c89b95bfb2 100644 --- a/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java +++ b/ingestion/src/main/java/feast/ingestion/transform/WriteToStore.java @@ -68,8 +68,8 @@ public abstract class WriteToStore extends PTransform, P public static final String METRIC_NAMESPACE = "WriteToStore"; public static final String ELEMENTS_WRITTEN_METRIC = "elements_written"; - private static final Counter elementsWritten = Metrics - .counter(METRIC_NAMESPACE, ELEMENTS_WRITTEN_METRIC); + private static final Counter elementsWritten = + Metrics.counter(METRIC_NAMESPACE, ELEMENTS_WRITTEN_METRIC); public abstract Store getStore(); @@ -155,32 +155,35 @@ public void processElement(ProcessContext context) { break; case CASSANDRA: CassandraConfig cassandraConfig = getStore().getCassandraConfig(); - SerializableFunction mapperFactory = new CassandraMutationMapperFactory(CassandraMutation.class); + SerializableFunction mapperFactory = + new CassandraMutationMapperFactory(CassandraMutation.class); input .apply( "Create CassandraMutation from FeatureRow", - ParDo.of(new FeatureRowToCassandraMutationDoFn( - getFeatureSetSpecs(), cassandraConfig.getDefaultTtl())) - ) + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + getFeatureSetSpecs(), cassandraConfig.getDefaultTtl()))) .apply( CassandraIO.write() .withHosts(Arrays.asList(cassandraConfig.getBootstrapHosts().split(","))) .withPort(cassandraConfig.getPort()) .withKeyspace(cassandraConfig.getKeyspace()) .withEntity(CassandraMutation.class) - .withMapperFactoryFn(mapperFactory) - ); + .withMapperFactoryFn(mapperFactory)); break; default: log.error("Store type '{}' is not supported. No Feature Row will be written.", storeType); break; } - input.apply("IncrementWriteToStoreElementsWrittenCounter", - MapElements.into(TypeDescriptors.booleans()).via((FeatureRow row) -> { - elementsWritten.inc(); - return true; - })); + input.apply( + "IncrementWriteToStoreElementsWrittenCounter", + MapElements.into(TypeDescriptors.booleans()) + .via( + (FeatureRow row) -> { + elementsWritten.inc(); + return true; + })); return PDone.in(input.getPipeline()); } diff --git a/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java b/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java index b92696c66d..3478227165 100644 --- a/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java +++ b/ingestion/src/main/java/feast/ingestion/utils/StoreUtil.java @@ -303,7 +303,7 @@ public static void setupCassandra(CassandraConfig cassandraConfig) { .ifNotExists() .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) .addClusteringColumn(CassandraMutation.FEATURE, DataType.text()) - .addStaticColumn(CassandraMutation.VALUE, DataType.blob()); + .addColumn(CassandraMutation.VALUE, DataType.blob()); log.info("Create Cassandra table if not exists.."); session.execute(createTable); diff --git a/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java b/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java index 0292735c58..177a8703e6 100644 --- a/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java +++ b/ingestion/src/main/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFn.java @@ -1,3 +1,19 @@ +/* + * 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.cassandra; import com.google.protobuf.Duration; @@ -18,8 +34,8 @@ public class FeatureRowToCassandraMutationDoFn extends DoFn { - private static final Logger log = org.slf4j.LoggerFactory - .getLogger(FeatureRowToCassandraMutationDoFn.class); + private static final Logger log = + org.slf4j.LoggerFactory.getLogger(FeatureRowToCassandraMutationDoFn.class); private Map featureSetSpecs; private Map maxAges; @@ -36,16 +52,16 @@ public FeatureRowToCassandraMutationDoFn(Map specs, Dura } } - /** - * Output a Cassandra mutation object for every feature in the feature row. - */ + /** Output a Cassandra mutation object for every feature in the feature row. */ @ProcessElement public void processElement(ProcessContext context) { FeatureRow featureRow = context.element(); try { FeatureSetSpec featureSetSpec = featureSetSpecs.get(featureRow.getFeatureSet()); - Set featureNames = featureSetSpec.getFeaturesList().stream() - .map(FeatureSpec::getName).collect(Collectors.toSet()); + Set featureNames = + featureSetSpec.getFeaturesList().stream() + .map(FeatureSpec::getName) + .collect(Collectors.toSet()); String key = CassandraMutation.keyFromFeatureRow(featureSetSpec, featureRow); Collection mutations = new ArrayList<>(); @@ -57,9 +73,7 @@ public void processElement(ProcessContext context) { field.getName(), ByteBuffer.wrap(field.getValue().toByteArray()), Timestamps.toMicros(featureRow.getEventTimestamp()), - maxAges.get(featureRow.getFeatureSet()) - ) - ); + maxAges.get(featureRow.getFeatureSet()))); } } @@ -67,7 +81,5 @@ public void processElement(ProcessContext context) { } catch (Exception e) { log.error(e.getMessage(), e); } - } - -} \ No newline at end of file +} diff --git a/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java b/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java index 4d66be089c..da914ca175 100644 --- a/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java +++ b/ingestion/src/test/java/feast/ingestion/transform/CassandraWriteToStoreIT.java @@ -66,13 +66,7 @@ public Store getStore() { return Store.newBuilder() .setType(StoreType.CASSANDRA) .setName("SERVING") - .setCassandraConfig( - CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setTableName("feature_store") - .setKeyspace("test") - .build()) + .setCassandraConfig(getCassandraConfig()) .build(); } @@ -86,10 +80,26 @@ public Map getFeatureSetSpecs() { } } + private static CassandraConfig getCassandraConfig() { + return CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setTableName("feature_store") + .setKeyspace("test") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "SimpleStrategy"); + put("replication_factor", "1"); + } + }) + .build(); + } + @BeforeClass public static void startServer() throws InterruptedException, IOException, TTransportException { LocalCassandra.start(); - LocalCassandra.createKeyspaceAndTable(); + LocalCassandra.createKeyspaceAndTable(getCassandraConfig()); } @Before @@ -120,7 +130,7 @@ public void setUp() { put("entity1", TestUtil.intValue(1)); put("entity2", TestUtil.strValue("a")); put("feature1", TestUtil.intValue(1)); - put("feature2", TestUtil.intValue(1)); + put("feature2", TestUtil.intValue(2)); } }); } @@ -146,7 +156,7 @@ public void testWriteCassandra_happyPath() throws InvalidProtocolBufferException List expectedFields = Arrays.asList( Field.newBuilder().setName("feature1").setValue(TestUtil.intValue(1)).build(), - Field.newBuilder().setName("feature2").setValue(TestUtil.intValue(1)).build()); + Field.newBuilder().setName("feature2").setValue(TestUtil.intValue(2)).build()); assertTrue(actualResults.containsAll(expectedFields)); assertEquals(expectedFields.size(), actualResults.size()); @@ -201,7 +211,7 @@ public void testWriteCassandra_shouldNotOverrideNewerValues() put("entity1", TestUtil.intValue(1)); put("entity2", TestUtil.strValue("a")); put("feature1", TestUtil.intValue(3)); - put("feature2", TestUtil.intValue(3)); + put("feature2", TestUtil.intValue(4)); } }); @@ -217,7 +227,7 @@ public void testWriteCassandra_shouldNotOverrideNewerValues() List expectedFields = Arrays.asList( Field.newBuilder().setName("feature1").setValue(TestUtil.intValue(1)).build(), - Field.newBuilder().setName("feature2").setValue(TestUtil.intValue(1)).build()); + Field.newBuilder().setName("feature2").setValue(TestUtil.intValue(2)).build()); assertTrue(actualResults.containsAll(expectedFields)); assertEquals(expectedFields.size(), actualResults.size()); diff --git a/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java index 1f1d11d72c..8c6874ecac 100644 --- a/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java +++ b/ingestion/src/test/java/feast/ingestion/util/CassandraStoreUtilIT.java @@ -1,3 +1,19 @@ +/* + * 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.ingestion.util; import com.datastax.driver.core.DataType; @@ -18,7 +34,7 @@ import org.junit.Test; public class CassandraStoreUtilIT { - + @BeforeClass public static void startServer() throws InterruptedException, IOException, TTransportException { LocalCassandra.start(); @@ -31,30 +47,35 @@ public void teardown() { @Test public void setupCassandra_shouldCreateKeyspaceAndTable() { - CassandraConfig config = CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("feature_store") - .putAllReplicationOptions(new HashMap() {{ - put("class", "NetworkTopologyStrategy"); - put("dc1", "2"); - put("dc2", "3"); - }}) - .build(); + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + } + }) + .build(); StoreUtil.setupCassandra(config); - Map actualReplication = LocalCassandra.getCluster() - .getMetadata() - .getKeyspace("test") - .getReplication(); - Map expectedReplication = new HashMap() {{ - put("class", "org.apache.cassandra.locator.NetworkTopologyStrategy"); - put("dc1", "2"); - put("dc2", "3"); - }}; - TableMetadata tableMetadata = LocalCassandra.getCluster() - .getMetadata().getKeyspace("test").getTable("feature_store"); + Map actualReplication = + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getReplication(); + Map expectedReplication = + new HashMap() { + { + put("class", "org.apache.cassandra.locator.NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + } + }; + TableMetadata tableMetadata = + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store"); Assert.assertEquals(expectedReplication, actualReplication); Assert.assertNotNull(tableMetadata); @@ -62,70 +83,85 @@ public void setupCassandra_shouldCreateKeyspaceAndTable() { @Test public void setupCassandra_shouldBeIdempotent_whenTableAlreadyExistsAndSchemaMatches() { - LocalCassandra.createKeyspaceAndTable(); + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "SimpleStrategy"); + put("replication_factor", "2"); + } + }) + .build(); + + LocalCassandra.createKeyspaceAndTable(config); // Check table is created - Assert.assertNotNull(LocalCassandra.getCluster() - .getMetadata().getKeyspace("test").getTable("feature_store")); - - CassandraConfig config = CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("feature_store") - .putAllReplicationOptions(new HashMap() {{ - put("class", "SimpleStrategy"); - put("replication_factor", "2"); - }}) - .build(); + Assert.assertNotNull( + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store")); StoreUtil.setupCassandra(config); - Assert.assertNotNull(LocalCassandra.getCluster() - .getMetadata().getKeyspace("test").getTable("feature_store")); + Assert.assertNotNull( + LocalCassandra.getCluster().getMetadata().getKeyspace("test").getTable("feature_store")); } @Test(expected = RuntimeException.class) public void setupCassandra_shouldThrowException_whenTableNameDoesNotMatchObjectMapper() { - CassandraConfig config = CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("test_data_store") - .putAllReplicationOptions(new HashMap() {{ - put("class", "NetworkTopologyStrategy"); - put("dc1", "2"); - put("dc2", "3"); - }}) - .build(); + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("test_data_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "NetworkTopologyStrategy"); + put("dc1", "2"); + put("dc2", "3"); + } + }) + .build(); StoreUtil.setupCassandra(config); } @Test(expected = RuntimeException.class) public void setupCassandra_shouldThrowException_whenTableSchemaDoesNotMatchObjectMapper() { - LocalCassandra.getSession().execute("CREATE KEYSPACE test " - + "WITH REPLICATION = {" - + "'class': 'SimpleStrategy', 'replication_factor': 2 }"); - - Create createTable = SchemaBuilder.createTable("test", "feature_store") - .ifNotExists() - .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) - .addClusteringColumn("featureName", DataType.text()) // Column name does not match in CassandraMutation - .addStaticColumn(CassandraMutation.VALUE, DataType.blob()); + LocalCassandra.getSession() + .execute( + "CREATE KEYSPACE test " + + "WITH REPLICATION = {" + + "'class': 'SimpleStrategy', 'replication_factor': 2 }"); + + Create createTable = + SchemaBuilder.createTable("test", "feature_store") + .ifNotExists() + .addPartitionKey(CassandraMutation.ENTITIES, DataType.text()) + .addClusteringColumn( + "featureName", DataType.text()) // Column name does not match in CassandraMutation + .addColumn(CassandraMutation.VALUE, DataType.blob()); LocalCassandra.getSession().execute(createTable); - CassandraConfig config = CassandraConfig.newBuilder() - .setBootstrapHosts(LocalCassandra.getHost()) - .setPort(LocalCassandra.getPort()) - .setKeyspace("test") - .setTableName("feature_store") - .putAllReplicationOptions(new HashMap() {{ - put("class", "SimpleStrategy"); - put("replication_factor", "2"); - }}) - .build(); + CassandraConfig config = + CassandraConfig.newBuilder() + .setBootstrapHosts(LocalCassandra.getHost()) + .setPort(LocalCassandra.getPort()) + .setKeyspace("test") + .setTableName("feature_store") + .putAllReplicationOptions( + new HashMap() { + { + put("class", "SimpleStrategy"); + put("replication_factor", "2"); + } + }) + .build(); StoreUtil.setupCassandra(config); } - } diff --git a/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java b/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java index a4680408a5..412c6488f2 100644 --- a/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java +++ b/ingestion/src/test/java/feast/store/serving/cassandra/FeatureRowToCassandraMutationDoFnTest.java @@ -1,3 +1,19 @@ +/* + * 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.cassandra; import com.google.protobuf.Duration; @@ -19,35 +35,60 @@ public class FeatureRowToCassandraMutationDoFnTest implements Serializable { - @Rule - public transient TestPipeline testPipeline = TestPipeline.create(); + @Rule public transient TestPipeline testPipeline = TestPipeline.create(); @Test public void processElement_shouldCreateCassandraMutation_givenFeatureRow() { - FeatureSetSpec featureSetSpec = TestUtil.createFeatureSetSpec("fs", 1, 10, - new HashMap() {{ put("entity1", Enum.INT64); }}, - new HashMap() {{ put("feature1", Enum.STRING); }}); - FeatureRow featureRow = TestUtil.createFeatureRow(featureSetSpec, 10, - new HashMap() {{ - put("entity1", TestUtil.intValue(1)); put("feature1", TestUtil.strValue("a")); }}); + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 10, + new HashMap() { + { + put("entity1", Enum.INT64); + } + }, + new HashMap() { + { + put("feature1", Enum.STRING); + } + }); + FeatureRow featureRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("feature1", TestUtil.strValue("a")); + } + }); PCollection input = testPipeline.apply(Create.of(featureRow)); PCollection output = - input.apply(ParDo.of(new FeatureRowToCassandraMutationDoFn( - new HashMap() {{ - put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); - }}, Duration.newBuilder().setSeconds(0).build()))); - - CassandraMutation[] expected = new CassandraMutation[] { - new CassandraMutation( - "fs:1:entity1=1", - "feature1", - ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), - 10000000, - 10 - ) - }; + input.apply( + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + new HashMap() { + { + put( + featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), + featureSetSpec); + } + }, + Duration.newBuilder().setSeconds(0).build()))); + + CassandraMutation[] expected = + new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 10) + }; PAssert.that(output).containsInAnyOrder(expected); @@ -55,42 +96,68 @@ public void processElement_shouldCreateCassandraMutation_givenFeatureRow() { } @Test - public void processElement_shouldCreateCassandraMutations_givenFeatureRowWithMultipleEntitiesAndFeatures() { - FeatureSetSpec featureSetSpec = TestUtil.createFeatureSetSpec("fs", 1, 10, - new HashMap() {{ put("entity1", Enum.INT64); put("entity2", Enum.STRING); }}, - new HashMap() {{ put("feature1", Enum.STRING); put("feature2", Enum.INT64); }}); - FeatureRow featureRow = TestUtil.createFeatureRow(featureSetSpec, 10, - new HashMap() {{ - put("entity1", TestUtil.intValue(1)); - put("entity2", TestUtil.strValue("b")); - put("feature1", TestUtil.strValue("a")); - put("feature2", TestUtil.intValue(2)); - }}); + public void + processElement_shouldCreateCassandraMutations_givenFeatureRowWithMultipleEntitiesAndFeatures() { + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 10, + new HashMap() { + { + put("entity1", Enum.INT64); + put("entity2", Enum.STRING); + } + }, + new HashMap() { + { + put("feature1", Enum.STRING); + put("feature2", Enum.INT64); + } + }); + FeatureRow featureRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("entity2", TestUtil.strValue("b")); + put("feature1", TestUtil.strValue("a")); + put("feature2", TestUtil.intValue(2)); + } + }); PCollection input = testPipeline.apply(Create.of(featureRow)); PCollection output = - input.apply(ParDo.of(new FeatureRowToCassandraMutationDoFn( - new HashMap() {{ - put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); - }}, Duration.newBuilder().setSeconds(0).build()))); - - CassandraMutation[] expected = new CassandraMutation[] { - new CassandraMutation( - "fs:1:entity1=1|entity2=b", - "feature1", - ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), - 10000000, - 10 - ), - new CassandraMutation( - "fs:1:entity1=1|entity2=b", - "feature2", - ByteBuffer.wrap(TestUtil.intValue(2).toByteArray()), - 10000000, - 10 - ) - }; + input.apply( + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + new HashMap() { + { + put( + featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), + featureSetSpec); + } + }, + Duration.newBuilder().setSeconds(0).build()))); + + CassandraMutation[] expected = + new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1|entity2=b", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 10), + new CassandraMutation( + "fs:1:entity1=1|entity2=b", + "feature2", + ByteBuffer.wrap(TestUtil.intValue(2).toByteArray()), + 10000000, + 10) + }; PAssert.that(output).containsInAnyOrder(expected); @@ -100,34 +167,59 @@ public void processElement_shouldCreateCassandraMutations_givenFeatureRowWithMul @Test public void processElement_shouldUseDefaultMaxAge_whenMissingMaxAge() { Duration defaultTtl = Duration.newBuilder().setSeconds(500).build(); - FeatureSetSpec featureSetSpec = TestUtil.createFeatureSetSpec("fs", 1, 0, - new HashMap() {{ put("entity1", Enum.INT64); }}, - new HashMap() {{ put("feature1", Enum.STRING); }}); - FeatureRow featureRow = TestUtil.createFeatureRow(featureSetSpec, 10, - new HashMap() {{ - put("entity1", TestUtil.intValue(1)); put("feature1", TestUtil.strValue("a")); }}); + FeatureSetSpec featureSetSpec = + TestUtil.createFeatureSetSpec( + "fs", + 1, + 0, + new HashMap() { + { + put("entity1", Enum.INT64); + } + }, + new HashMap() { + { + put("feature1", Enum.STRING); + } + }); + FeatureRow featureRow = + TestUtil.createFeatureRow( + featureSetSpec, + 10, + new HashMap() { + { + put("entity1", TestUtil.intValue(1)); + put("feature1", TestUtil.strValue("a")); + } + }); PCollection input = testPipeline.apply(Create.of(featureRow)); PCollection output = - input.apply(ParDo.of(new FeatureRowToCassandraMutationDoFn( - new HashMap() {{ - put(featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), featureSetSpec); - }}, defaultTtl))); - - CassandraMutation[] expected = new CassandraMutation[] { - new CassandraMutation( - "fs:1:entity1=1", - "feature1", - ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), - 10000000, - 500 - ) - }; + input.apply( + ParDo.of( + new FeatureRowToCassandraMutationDoFn( + new HashMap() { + { + put( + featureSetSpec.getName() + ":" + featureSetSpec.getVersion(), + featureSetSpec); + } + }, + defaultTtl))); + + CassandraMutation[] expected = + new CassandraMutation[] { + new CassandraMutation( + "fs:1:entity1=1", + "feature1", + ByteBuffer.wrap(TestUtil.strValue("a").toByteArray()), + 10000000, + 500) + }; PAssert.that(output).containsInAnyOrder(expected); testPipeline.run(); } - } diff --git a/ingestion/src/test/java/feast/test/TestUtil.java b/ingestion/src/test/java/feast/test/TestUtil.java index 0fcb9f4e16..51d4712759 100644 --- a/ingestion/src/test/java/feast/test/TestUtil.java +++ b/ingestion/src/test/java/feast/test/TestUtil.java @@ -24,7 +24,9 @@ import feast.core.FeatureSetProto.EntitySpec; import feast.core.FeatureSetProto.FeatureSetSpec; import feast.core.FeatureSetProto.FeatureSpec; +import feast.core.StoreProto.Store.CassandraConfig; import feast.ingestion.transform.WriteToStore; +import feast.ingestion.utils.StoreUtil; import feast.storage.RedisProto.RedisKey; import feast.types.FeatureRowProto.FeatureRow; import feast.types.FeatureRowProto.FeatureRow.Builder; @@ -64,7 +66,6 @@ import org.apache.thrift.transport.TTransportException; import org.apache.zookeeper.server.ServerConfig; import org.apache.zookeeper.server.ZooKeeperServerMain; -import org.cassandraunit.dataset.cql.ClassPathCQLDataSet; import org.cassandraunit.utils.EmbeddedCassandraServerHelper; import org.joda.time.Duration; import redis.embedded.RedisServer; @@ -100,10 +101,8 @@ public static void start() throws InterruptedException, IOException, TTransportE EmbeddedCassandraServerHelper.startEmbeddedCassandra(); } - public static void createKeyspaceAndTable() { - new ClassPathCQLDataSet("embedded-store/LoadCassandra.cql", true, true) - .getCQLStatements() - .forEach(s -> LocalCassandra.getSession().execute(s)); + public static void createKeyspaceAndTable(CassandraConfig config) { + StoreUtil.setupCassandra(config); } public static String getHost() { diff --git a/ingestion/src/test/resources/embedded-store/LoadCassandra.cql b/ingestion/src/test/resources/embedded-store/LoadCassandra.cql deleted file mode 100644 index c80da294b7..0000000000 --- a/ingestion/src/test/resources/embedded-store/LoadCassandra.cql +++ /dev/null @@ -1,8 +0,0 @@ -CREATE KEYSPACE test with replication = {'class':'SimpleStrategy','replication_factor':1}; - -CREATE TABLE test.feature_store( - entities text, - feature text, - value blob, - PRIMARY KEY (entities, feature) -) WITH CLUSTERING ORDER BY (feature DESC); \ No newline at end of file diff --git a/protos/feast/core/Store.proto b/protos/feast/core/Store.proto index 34254cf383..9e1dc33143 100644 --- a/protos/feast/core/Store.proto +++ b/protos/feast/core/Store.proto @@ -110,6 +110,8 @@ message Store { // // Columns: // - entities: concatenated string of feature set name and all entities' keys and values + // entities concatenated format - [feature_set]:[entity_name1=entity_value1]|[entity_name2=entity_value2] + // TODO: string representation of float or double types may have different value in different runtime or platform // - feature: clustering column where each feature is a column // - value: byte array of Value (refer to feast.types.Value) // @@ -135,8 +137,15 @@ message Store { string bootstrap_hosts = 1; int32 port = 2; string keyspace = 3; + + // Please note that table name must be "feature_store" as is specified in the @Table annotation of the + // datastax object mapper string table_name = 4; + + // This specifies the replication strategy to use. Please refer to docs for more details: + // https://docs.datastax.com/en/dse/6.7/cql/cql/cql_reference/cql_commands/cqlCreateKeyspace.html#cqlCreateKeyspace__cqlCreateKeyspacereplicationmap-Pr3yUQ7t map replication_options = 5; + // Default expiration in seconds to use when FeatureSetSpec does not have max_age defined. // Specify 0 for no default expiration google.protobuf.Duration default_ttl = 6;