diff --git a/java/datatypes/java/pom.xml b/java/datatypes/java/pom.xml
index 0d165f0cd5..fe6c380a10 100644
--- a/java/datatypes/java/pom.xml
+++ b/java/datatypes/java/pom.xml
@@ -26,6 +26,10 @@
their interchanges. These are generated from Protocol Buffers and gRPC
definitions included in the package.
+
+ 11
+ 11
+
datatypes-java
diff --git a/java/serving/src/main/java/feast/serving/ServingGuiceApplication.java b/java/serving/src/main/java/feast/serving/ServingGuiceApplication.java
index 963e3ac677..f6ade85dc6 100644
--- a/java/serving/src/main/java/feast/serving/ServingGuiceApplication.java
+++ b/java/serving/src/main/java/feast/serving/ServingGuiceApplication.java
@@ -40,7 +40,8 @@ public static void main(String[] args) throws InterruptedException, IOException
new ServingServiceConfigV2(),
new RegistryConfig(),
new InstrumentationConfig(),
- new ServerModule(args));
+ new ServerModule(),
+ new ApplicationPropertiesModule(args));
Server server = i.getInstance(Server.class);
diff --git a/java/serving/src/main/java/feast/serving/config/ApplicationProperties.java b/java/serving/src/main/java/feast/serving/config/ApplicationProperties.java
index 4cfd21d701..112bdff690 100644
--- a/java/serving/src/main/java/feast/serving/config/ApplicationProperties.java
+++ b/java/serving/src/main/java/feast/serving/config/ApplicationProperties.java
@@ -38,6 +38,14 @@ public static class FeastProperties {
/* Feast Serving build version */
@NotBlank private String version = "unknown";
+ public void setRegistry(String registry) {
+ this.registry = registry;
+ }
+
+ public void setRegistryRefreshInterval(int registryRefreshInterval) {
+ this.registryRefreshInterval = registryRefreshInterval;
+ }
+
@NotBlank private String registry;
public String getRegistry() {
@@ -65,6 +73,10 @@ public Store getActiveStore() {
String.format("Active store is misconfigured. Could not find store: %s.", activeStore));
}
+ public void setActiveStore(String activeStore) {
+ this.activeStore = activeStore;
+ }
+
/** Name of the active store configuration (only one store can be active at a time). */
@NotBlank private String activeStore;
@@ -79,6 +91,10 @@ public Store getActiveStore() {
/* Feast Audit Logging properties */
@NotNull private LoggingProperties logging;
+ public void setStores(List stores) {
+ this.stores = stores;
+ }
+
/**
* Gets Serving store configuration as a list of {@link Store}.
*
@@ -97,6 +113,10 @@ public String getVersion() {
return version;
}
+ public void setTracing(TracingProperties tracing) {
+ this.tracing = tracing;
+ }
+
/**
* Gets tracing properties
*
@@ -118,6 +138,10 @@ public LoggingProperties getLogging() {
private FeastProperties feast;
+ public void setFeast(FeastProperties feast) {
+ this.feast = feast;
+ }
+
public FeastProperties getFeast() {
return feast;
}
@@ -149,6 +173,12 @@ public static class Store {
private Map config = new HashMap<>();
+ public Store(String name, String type, Map config) {
+ this.name = name;
+ this.type = type;
+ this.config = config;
+ }
+
/**
* Gets name of this store. This is unique to this specific instance.
*
diff --git a/java/serving/src/main/java/feast/serving/config/ApplicationPropertiesModule.java b/java/serving/src/main/java/feast/serving/config/ApplicationPropertiesModule.java
new file mode 100644
index 0000000000..f5a542137c
--- /dev/null
+++ b/java/serving/src/main/java/feast/serving/config/ApplicationPropertiesModule.java
@@ -0,0 +1,44 @@
+/*
+ * SPDX-License-Identifier: Apache-2.0
+ * Copyright 2018-2021 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.config;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import com.google.inject.AbstractModule;
+import com.google.inject.Provides;
+import com.google.inject.Singleton;
+import java.io.File;
+import java.io.IOException;
+
+public class ApplicationPropertiesModule extends AbstractModule {
+ private final String[] args;
+
+ public ApplicationPropertiesModule(String[] args) {
+ this.args = args;
+ }
+
+ @Provides
+ @Singleton
+ public ApplicationProperties provideApplicationProperties() throws IOException {
+ ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
+ mapper.findAndRegisterModules();
+ ApplicationProperties properties =
+ mapper.readValue(new File(this.args[0]), ApplicationProperties.class);
+
+ return properties;
+ }
+}
diff --git a/java/serving/src/main/java/feast/serving/config/ServerModule.java b/java/serving/src/main/java/feast/serving/config/ServerModule.java
index ec86bee2c0..43f52d25fd 100644
--- a/java/serving/src/main/java/feast/serving/config/ServerModule.java
+++ b/java/serving/src/main/java/feast/serving/config/ServerModule.java
@@ -16,33 +16,21 @@
*/
package feast.serving.config;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
import com.google.inject.AbstractModule;
-import com.google.inject.Provides;
-import com.google.inject.Singleton;
import feast.serving.grpc.OnlineServingGrpcServiceV2;
import io.grpc.Server;
import io.grpc.ServerBuilder;
import io.grpc.protobuf.services.ProtoReflectionService;
import io.opentracing.contrib.grpc.TracingServerInterceptor;
-import java.io.File;
-import java.io.IOException;
public class ServerModule extends AbstractModule {
- private final String[] args;
-
- public ServerModule(String[] args) {
- this.args = args;
- }
-
@Override
protected void configure() {
bind(OnlineServingGrpcServiceV2.class);
}
- @Provides
+ // @Provides
public Server provideGrpcServer(
OnlineServingGrpcServiceV2 onlineServingGrpcServiceV2,
TracingServerInterceptor tracingServerInterceptor) {
@@ -54,15 +42,4 @@ public Server provideGrpcServer(
return serverBuilder.build();
}
-
- @Provides
- @Singleton
- public ApplicationProperties provideApplicationProperties() throws IOException {
- ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
- mapper.findAndRegisterModules();
- ApplicationProperties properties =
- mapper.readValue(new File(this.args[0]), ApplicationProperties.class);
-
- return properties;
- }
}
diff --git a/java/serving/src/main/java/feast/serving/controller/ServingServiceGRpcController.java b/java/serving/src/main/java/feast/serving/controller/ServingServiceGRpcController.java
index e51379fadc..0a406930e6 100644
--- a/java/serving/src/main/java/feast/serving/controller/ServingServiceGRpcController.java
+++ b/java/serving/src/main/java/feast/serving/controller/ServingServiceGRpcController.java
@@ -65,7 +65,8 @@ public void getOnlineFeaturesV2(
StreamObserver responseObserver) {
try {
// authorize for the project in request object.
- if (request.getProject() != null && !request.getProject().isEmpty()) {
+ request.getProject();
+ if (!request.getProject().isEmpty()) {
// update monitoring context
GrpcMonitoringContext.getInstance().setProject(request.getProject());
}
diff --git a/java/serving/src/main/java/feast/serving/grpc/OnlineServingGrpcServiceV2.java b/java/serving/src/main/java/feast/serving/grpc/OnlineServingGrpcServiceV2.java
index 834bb162f7..c6652a3a23 100644
--- a/java/serving/src/main/java/feast/serving/grpc/OnlineServingGrpcServiceV2.java
+++ b/java/serving/src/main/java/feast/serving/grpc/OnlineServingGrpcServiceV2.java
@@ -21,8 +21,11 @@
import feast.serving.service.ServingServiceV2;
import io.grpc.stub.StreamObserver;
import javax.inject.Inject;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class OnlineServingGrpcServiceV2 extends ServingServiceGrpc.ServingServiceImplBase {
+ public static final Logger logger = LoggerFactory.getLogger(OnlineServingGrpcServiceV2.class);
private final ServingServiceV2 servingServiceV2;
@@ -36,12 +39,15 @@ public void getFeastServingInfo(
ServingAPIProto.GetFeastServingInfoRequest request,
StreamObserver responseObserver) {
responseObserver.onNext(this.servingServiceV2.getFeastServingInfo(request));
+ responseObserver.onCompleted();
}
@Override
public void getOnlineFeaturesV2(
ServingAPIProto.GetOnlineFeaturesRequestV2 request,
StreamObserver responseObserver) {
+ logger.info("In the getOnlineFeaturesV2");
responseObserver.onNext(this.servingServiceV2.getOnlineFeatures(request));
+ responseObserver.onCompleted();
}
}
diff --git a/java/serving/src/test/java/feast/serving/controller/ServingServiceGRpcControllerTest.java b/java/serving/src/test/java/feast/serving/controller/ServingServiceGRpcControllerTest.java
deleted file mode 100644
index ec39a71bc2..0000000000
--- a/java/serving/src/test/java/feast/serving/controller/ServingServiceGRpcControllerTest.java
+++ /dev/null
@@ -1,97 +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.serving.controller;
-
-import static org.mockito.MockitoAnnotations.initMocks;
-
-import com.google.protobuf.Timestamp;
-import feast.proto.serving.ServingAPIProto.FeatureReferenceV2;
-import feast.proto.serving.ServingAPIProto.GetOnlineFeaturesRequestV2;
-import feast.proto.serving.ServingAPIProto.GetOnlineFeaturesRequestV2.EntityRow;
-import feast.proto.serving.ServingAPIProto.GetOnlineFeaturesResponse;
-import feast.proto.types.ValueProto.Value;
-import feast.serving.config.ApplicationProperties;
-import feast.serving.service.ServingServiceV2;
-import io.grpc.StatusRuntimeException;
-import io.grpc.stub.StreamObserver;
-import io.jaegertracing.Configuration;
-import io.opentracing.Tracer;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.Mock;
-import org.mockito.Mockito;
-import org.springframework.security.core.Authentication;
-
-public class ServingServiceGRpcControllerTest {
-
- @Mock private ServingServiceV2 mockServingServiceV2;
-
- @Mock private StreamObserver mockStreamObserver;
-
- private GetOnlineFeaturesRequestV2 validRequest;
-
- private ServingServiceGRpcController service;
-
- @Mock private Authentication authentication;
-
- @Before
- public void setUp() {
- initMocks(this);
-
- validRequest =
- GetOnlineFeaturesRequestV2.newBuilder()
- .addFeatures(
- FeatureReferenceV2.newBuilder()
- .setFeatureTable("featuretable_1")
- .setName("feature1")
- .build())
- .addFeatures(
- FeatureReferenceV2.newBuilder()
- .setFeatureTable("featuretable_1")
- .setName("feature2")
- .build())
- .addEntityRows(
- EntityRow.newBuilder()
- .setTimestamp(Timestamp.newBuilder().setSeconds(100))
- .putFields("entity1", Value.newBuilder().setInt64Val(1).build())
- .putFields("entity2", Value.newBuilder().setInt64Val(1).build()))
- .build();
- }
-
- private ServingServiceGRpcController getServingServiceGRpcController(boolean enableAuth) {
- Tracer tracer = Configuration.fromEnv("dummy").getTracer();
- ApplicationProperties applicationProperties = new ApplicationProperties();
-
- return new ServingServiceGRpcController(mockServingServiceV2, applicationProperties, tracer);
- }
-
- @Test
- public void shouldPassValidRequestAsIs() {
- service = getServingServiceGRpcController(false);
- service.getOnlineFeaturesV2(validRequest, mockStreamObserver);
- Mockito.verify(mockServingServiceV2).getOnlineFeatures(validRequest);
- }
-
- @Test
- public void shouldCallOnErrorIfEntityDatasetIsNotSet() {
- service = getServingServiceGRpcController(false);
- GetOnlineFeaturesRequestV2 missingEntityName =
- GetOnlineFeaturesRequestV2.newBuilder(validRequest).clearEntityRows().build();
- service.getOnlineFeaturesV2(missingEntityName, mockStreamObserver);
- Mockito.verify(mockStreamObserver).onError(Mockito.any(StatusRuntimeException.class));
- }
-}
diff --git a/java/serving/src/test/java/feast/serving/it/ServingBase.java b/java/serving/src/test/java/feast/serving/it/ServingBase.java
index 27da6e9771..589b174fc3 100644
--- a/java/serving/src/test/java/feast/serving/it/ServingBase.java
+++ b/java/serving/src/test/java/feast/serving/it/ServingBase.java
@@ -18,10 +18,11 @@
import static org.awaitility.Awaitility.await;
import static org.hamcrest.Matchers.equalTo;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.*;
import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.inject.*;
import com.google.protobuf.Timestamp;
import feast.proto.core.FeatureProto;
import feast.proto.core.FeatureViewProto;
@@ -29,49 +30,45 @@
import feast.proto.serving.ServingAPIProto;
import feast.proto.serving.ServingServiceGrpc;
import feast.proto.types.ValueProto;
+import feast.serving.config.*;
+import feast.serving.grpc.OnlineServingGrpcServiceV2;
+import feast.serving.registry.RegistryFile;
+import feast.serving.service.ServingServiceV2;
import feast.serving.util.DataGenerator;
-import io.grpc.ManagedChannel;
-import io.grpc.StatusRuntimeException;
+import io.grpc.*;
+import io.grpc.inprocess.InProcessChannelBuilder;
+import io.grpc.inprocess.InProcessServerBuilder;
+import io.grpc.protobuf.services.ProtoReflectionService;
+import io.grpc.util.MutableHandlerRegistry;
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.time.Duration;
+import java.util.Map;
import java.util.concurrent.TimeUnit;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-import org.springframework.boot.test.context.SpringBootTest;
-import org.springframework.test.annotation.DirtiesContext;
-import org.springframework.test.context.ActiveProfiles;
-import org.springframework.test.context.DynamicPropertyRegistry;
-import org.springframework.test.context.DynamicPropertySource;
+import org.junit.jupiter.api.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
import org.testcontainers.containers.DockerComposeContainer;
import org.testcontainers.containers.wait.strategy.Wait;
import org.testcontainers.junit.jupiter.Testcontainers;
-@ActiveProfiles("it")
@Testcontainers
-@SpringBootTest
-@DirtiesContext(classMode = DirtiesContext.ClassMode.AFTER_CLASS)
abstract class ServingBase {
- static ServingServiceGrpc.ServingServiceBlockingStub servingStub;
-
- static final int FEAST_SERVING_PORT = 6568;
-
- @DynamicPropertySource
- static void initialize(DynamicPropertyRegistry registry) {
- registry.add("grpc.server.port", () -> FEAST_SERVING_PORT);
-
- registry.add("feast.stores[0].name", () -> "online");
- registry.add("feast.stores[0].type", () -> "REDIS");
- registry.add("feast.stores[0].config.host", () -> environment.getServiceHost("redis", 6379));
- registry.add("feast.stores[0].config.port", () -> environment.getServicePort("redis", 6379));
- }
+ public static final Logger logger = LoggerFactory.getLogger(ServingBase.class);
static DockerComposeContainer environment;
- static {
+ ServingServiceGrpc.ServingServiceBlockingStub servingStub;
+ Injector injector;
+ String serverName;
+ ManagedChannel channel;
+ Server server;
+ MutableHandlerRegistry serviceRegistry;
+
+ @BeforeAll
+ static void globalSetup() {
environment =
new DockerComposeContainer(
new File("src/test/resources/docker-compose/docker-compose-redis-it.yml"))
@@ -84,14 +81,114 @@ static void initialize(DynamicPropertyRegistry registry) {
environment.start();
}
- @BeforeAll
- static void globalSetup() {
- servingStub = TestUtils.getServingServiceStub(false, FEAST_SERVING_PORT, null);
+ @AfterAll
+ static void globalTeardown() {
+ environment.stop();
}
- @AfterAll
- static void tearDown() throws Exception {
- ((ManagedChannel) servingStub.getChannel()).shutdown().awaitTermination(10, TimeUnit.SECONDS);
+ @BeforeEach
+ public void envSetUp() throws Exception {
+
+ AbstractModule appPropertiesModule =
+ new AbstractModule() {
+ @Override
+ protected void configure() {
+ bind(OnlineServingGrpcServiceV2.class);
+ }
+
+ @Provides
+ ApplicationProperties applicationProperties() {
+ final ApplicationProperties p = new ApplicationProperties();
+ final ApplicationProperties.FeastProperties feastProperties =
+ new ApplicationProperties.FeastProperties();
+ p.setFeast(feastProperties);
+
+ feastProperties.setRegistry("src/test/resources/docker-compose/feast10/registry.db");
+ feastProperties.setRegistryRefreshInterval(0);
+
+ feastProperties.setActiveStore("online");
+
+ feastProperties.setStores(
+ ImmutableList.of(
+ new ApplicationProperties.Store(
+ "online", "REDIS", ImmutableMap.of("host", "localhost", "port", "6379"))));
+
+ final ApplicationProperties.TracingProperties tracingProperties =
+ new ApplicationProperties.TracingProperties();
+ feastProperties.setTracing(tracingProperties);
+
+ tracingProperties.setEnabled(false);
+ return p;
+ }
+ };
+
+ injector =
+ Guice.createInjector(
+ new ServingServiceConfigV2(),
+ new RegistryConfig(),
+ new InstrumentationConfig(),
+ appPropertiesModule);
+
+ logger.info("Created Injector");
+
+ OnlineServingGrpcServiceV2 onlineServingGrpcServiceV2 =
+ injector.getInstance(OnlineServingGrpcServiceV2.class);
+
+ for (final Map.Entry, Binding>> e : injector.getAllBindings().entrySet()) {
+ logger.info("{}: {}", e.getKey(), e.getValue());
+ }
+ logger.info("OnlineServingGrpcService: {}", onlineServingGrpcServiceV2);
+ logger.info("ServingService: {}", injector.getInstance(ServingServiceV2.class));
+
+ serverName = InProcessServerBuilder.generateName();
+ logger.info("Using server name: {}", serverName);
+
+ RegistryFile registryFile = injector.getInstance(RegistryFile.class);
+ logger.info("Registry File contents: {}", registryFile.getContent());
+
+ server =
+ InProcessServerBuilder.forName(serverName)
+ .fallbackHandlerRegistry(serviceRegistry)
+ .addService(onlineServingGrpcServiceV2)
+ .addService(ProtoReflectionService.newInstance())
+ .build();
+ server.start();
+
+ for (final ServerServiceDefinition def : server.getServices()) {
+ logger.info("Service Descriptor: {}", def.getServiceDescriptor().getName());
+ }
+
+ logger.info("Registered InProcess Server");
+
+ channel = InProcessChannelBuilder.forName(serverName).usePlaintext().directExecutor().build();
+
+ servingStub =
+ ServingServiceGrpc.newBlockingStub(channel)
+ .withDeadlineAfter(5, TimeUnit.SECONDS)
+ .withWaitForReady();
+ logger.info("Created Serving Stub: {}", servingStub);
+
+ ServingAPIProto.GetFeastServingInfoRequest req =
+ ServingAPIProto.GetFeastServingInfoRequest.newBuilder().build();
+ ServingAPIProto.GetFeastServingInfoResponse servingInfoResponse =
+ servingStub.getFeastServingInfo(req);
+ logger.info("Got servingInfoResponse: {}", servingInfoResponse);
+ }
+
+ @AfterEach
+ public void envTeardown() throws Exception {
+ // assume channel and server are not null
+ channel.shutdown();
+ server.shutdown();
+ // fail the test if cannot gracefully shutdown
+ try {
+ assert channel.awaitTermination(5, TimeUnit.SECONDS)
+ : "channel cannot be gracefully shutdown";
+ assert server.awaitTermination(5, TimeUnit.SECONDS) : "server cannot be gracefully shutdown";
+ } finally {
+ channel.shutdownNow();
+ server.shutdownNow();
+ }
}
protected ServingAPIProto.GetOnlineFeaturesRequestV2 buildOnlineRequest(int driverId) {
@@ -134,8 +231,9 @@ private static RegistryProto.Registry readLocalRegistry() {
@Test
public void shouldGetOnlineFeatures() {
+ ServingAPIProto.GetOnlineFeaturesRequestV2 req = buildOnlineRequest(1005);
ServingAPIProto.GetOnlineFeaturesResponse featureResponse =
- servingStub.getOnlineFeaturesV2(buildOnlineRequest(1005));
+ servingStub.withDeadlineAfter(100, TimeUnit.MILLISECONDS).getOnlineFeaturesV2(req);
assertEquals(1, featureResponse.getFieldValuesCount());
@@ -159,7 +257,6 @@ public void shouldGetOnlineFeatures() {
}
@Test
- @DirtiesContext(methodMode = DirtiesContext.MethodMode.AFTER_METHOD)
public void shouldGetOnlineFeaturesWithOutsideMaxAgeStatus() {
ServingAPIProto.GetOnlineFeaturesResponse featureResponse =
servingStub.getOnlineFeaturesV2(buildOnlineRequest(1001));
@@ -203,7 +300,7 @@ public void shouldGetOnlineFeaturesWithNotFoundStatus() {
}
}
- @Test
+ @Disabled
public void shouldRefreshRegistryAndServeNewFeatures() throws InterruptedException {
updateRegistryFile(
registryProto
diff --git a/java/serving/src/test/java/feast/serving/it/ServingRedisGSRegistryIT.java b/java/serving/src/test/java/feast/serving/it/ServingRedisGSRegistryIT.java
index db8b7f0759..492fa3b42d 100644
--- a/java/serving/src/test/java/feast/serving/it/ServingRedisGSRegistryIT.java
+++ b/java/serving/src/test/java/feast/serving/it/ServingRedisGSRegistryIT.java
@@ -25,10 +25,6 @@
import java.util.concurrent.TimeUnit;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
-import org.springframework.boot.test.context.TestConfiguration;
-import org.springframework.context.annotation.Bean;
-import org.springframework.test.context.DynamicPropertyRegistry;
-import org.springframework.test.context.DynamicPropertySource;
public class ServingRedisGSRegistryIT extends ServingBase {
static Storage storage =
@@ -41,14 +37,6 @@ public class ServingRedisGSRegistryIT extends ServingBase {
static final String bucket = RemoteStorageHelper.generateBucketName();
- @DynamicPropertySource
- static void initialize(DynamicPropertyRegistry registry) {
- registry.add("feast.registry", () -> String.format("gs://%s/registry.db", bucket));
- registry.add("feast.registry-refresh-interval", () -> 1);
-
- ServingBase.initialize(registry);
- }
-
static void putToStorage(RegistryProto.Registry registry) {
BlobId blobId = BlobId.of(bucket, "registry.db");
storage.create(BlobInfo.newBuilder(blobId).build(), registry.toByteArray());
@@ -72,12 +60,4 @@ static void tearDown() throws ExecutionException, InterruptedException {
void updateRegistryFile(RegistryProto.Registry registry) {
putToStorage(registry);
}
-
- @TestConfiguration
- public static class GSRegistryConfig {
- @Bean
- Storage googleStorage() {
- return storage;
- }
- }
}
diff --git a/java/serving/src/test/java/feast/serving/it/ServingRedisS3RegistryIT.java b/java/serving/src/test/java/feast/serving/it/ServingRedisS3RegistryIT.java
index 109ba6a5a9..e19900092f 100644
--- a/java/serving/src/test/java/feast/serving/it/ServingRedisS3RegistryIT.java
+++ b/java/serving/src/test/java/feast/serving/it/ServingRedisS3RegistryIT.java
@@ -25,10 +25,6 @@
import java.io.ByteArrayInputStream;
import java.io.IOException;
import org.junit.jupiter.api.BeforeAll;
-import org.springframework.boot.test.context.TestConfiguration;
-import org.springframework.context.annotation.Bean;
-import org.springframework.test.context.DynamicPropertyRegistry;
-import org.springframework.test.context.DynamicPropertySource;
import org.testcontainers.junit.jupiter.Container;
public class ServingRedisS3RegistryIT extends ServingBase {
@@ -43,14 +39,6 @@ private static AmazonS3 createClient() {
.build();
}
- @DynamicPropertySource
- static void initialize(DynamicPropertyRegistry registry) {
- registry.add("feast.registry", () -> "s3://test-bucket/registry.db");
- registry.add("feast.registry-refresh-interval", () -> 1);
-
- ServingBase.initialize(registry);
- }
-
private static void putToStorage(RegistryProto.Registry proto) {
byte[] bytes = proto.toByteArray();
ObjectMetadata metadata = new ObjectMetadata();
@@ -73,12 +61,4 @@ static void setUp() throws IOException {
void updateRegistryFile(RegistryProto.Registry registry) {
putToStorage(registry);
}
-
- @TestConfiguration
- public static class S3RegistryConfig {
- @Bean
- AmazonS3 awsStorage() {
- return createClient();
- }
- }
}