From 2f5edc6a21ed41f28e1a5bd923f60edbbc557018 Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Mon, 4 Nov 2024 14:35:17 +0000 Subject: [PATCH 1/7] PD-257044-changes-for-databus --- .../auth/TableAuthIdentityManagerDAOTest.java | 9 +- .../auth/TableRoleManagerDAOTest.java | 9 +- .../integration/sor/CasStashTableTest.java | 4 +- .../core/kafka/KafkaProducerService.java | 8 +- .../bazaarvoice/emodb/sor/api/DataStore.java | 6 + .../com/bazaarvoice/emodb/sor/api/Names.java | 22 ++++ .../emodb/sor/api/UpdateRefModel.java | 63 ++++++++++ sor/pom.xml | 6 + .../emodb/sor/core/DefaultDataStore.java | 117 ++++++++++++------ .../sor/core/test/InMemoryDataStore.java | 15 ++- .../emodb/sor/core/CompactorTest.java | 6 +- .../emodb/sor/core/DataStoreTest.java | 9 +- .../emodb/sor/core/MinSplitSizeTest.java | 4 +- .../emodb/sor/core/RedundantDeltaTest.java | 16 +-- .../emodb/sor/core/SorUpdateTest.java | 5 +- .../emodb/sor/test/MultiDCDataStores.java | 8 +- .../table/db/astyanax/TableLifeCycleTest.java | 4 +- .../web/resources/sor/DataStoreResource1.java | 67 ++++++++-- .../emodb/web/purge/PurgeTest.java | 4 +- .../emodb/web/scanner/ScanUploaderTest.java | 7 +- .../DataStoreScanStatusDAOTest.java | 5 +- .../DataStoreStashRequestDAOTest.java | 5 +- .../web/settings/SettingsManagerTest.java | 4 +- 23 files changed, 313 insertions(+), 90 deletions(-) create mode 100644 sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/UpdateRefModel.java diff --git a/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java b/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java index c09cceaa38..67c3849a76 100644 --- a/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java +++ b/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java @@ -3,6 +3,8 @@ import com.bazaarvoice.emodb.auth.apikey.ApiKey; import com.bazaarvoice.emodb.auth.apikey.ApiKeyModification; import com.bazaarvoice.emodb.auth.identity.TableAuthIdentityManagerDAO; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; @@ -22,6 +24,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -38,7 +41,7 @@ public class TableAuthIdentityManagerDAOTest { */ @Test public void testRebuildIdIndex() { - DataStore dataStore = new InMemoryDataStore(new MetricRegistry()); + DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); Supplier idSupplier = () -> "id0"; TableAuthIdentityManagerDAO tableAuthIdentityManagerDAO = new TableAuthIdentityManagerDAO<>( ApiKey.class, dataStore, "__auth:keys", "__auth:internal_ids", "app_global:sys", @@ -76,7 +79,7 @@ public void testRebuildIdIndex() { @Test public void testGrandfatheredInId() { - DataStore dataStore = new InMemoryDataStore(new MetricRegistry()); + DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); Supplier idSupplier = () -> "id0"; TableAuthIdentityManagerDAO tableAuthIdentityManagerDAO = new TableAuthIdentityManagerDAO<>( ApiKey.class, dataStore, "__auth:keys", "__auth:internal_ids", "app_global:sys", @@ -128,7 +131,7 @@ public void testGrandfatheredInId() { @Test public void testIdAttributeCompatibility() { - DataStore dataStore = new InMemoryDataStore(new MetricRegistry()); + DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); Supplier idSupplier = () -> "id0"; TableAuthIdentityManagerDAO tableAuthIdentityManagerDAO = new TableAuthIdentityManagerDAO<>( ApiKey.class, dataStore, "__auth:keys", "__auth:internal_ids", "app_global:sys", diff --git a/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java b/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java index 6b02653a6c..0c54beb6b4 100644 --- a/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java +++ b/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java @@ -9,6 +9,8 @@ import com.bazaarvoice.emodb.auth.role.RoleModification; import com.bazaarvoice.emodb.auth.role.RoleNotFoundException; import com.bazaarvoice.emodb.auth.role.TableRoleManagerDAO; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; @@ -36,10 +38,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.reset; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.*; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -63,7 +62,7 @@ public class TableRoleManagerDAOTest { @BeforeMethod public void setUp() { // DataStore and PermissionManager are fairly heavy to fully mock. Use spies on in-memory implementations instead - _backendDataStore = new InMemoryDataStore(new MetricRegistry()); + _backendDataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); _dataStore = spy(_backendDataStore); _permissionResolver = new EmoPermissionResolver(null, null); _backendPermissionManager = new InMemoryPermissionManager(_permissionResolver); diff --git a/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java b/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java index a90855e41e..702cf97d40 100644 --- a/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java +++ b/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java @@ -10,6 +10,8 @@ import com.bazaarvoice.emodb.common.zookeeper.store.ValueStore; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.api.DataCenters; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.condition.Conditions; @@ -97,7 +99,7 @@ public void setup() throws Exception { _astyanaxTableDAO.setCQLStashTableDAO(cqlStashTableDAO); // Don't store table definitions in the actual backing store so as not to interrupt other tests. Use a // private in-memory implementation. - _tableBackingStore = new InMemoryDataStore(new MetricRegistry()); + _tableBackingStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); _astyanaxTableDAO.setBackingStore(_tableBackingStore); _lifeCycleRegistry.start(); diff --git a/queue/src/main/java/com/bazaarvoice/emodb/queue/core/kafka/KafkaProducerService.java b/queue/src/main/java/com/bazaarvoice/emodb/queue/core/kafka/KafkaProducerService.java index 1a7cc72eee..10f60a2eaa 100644 --- a/queue/src/main/java/com/bazaarvoice/emodb/queue/core/kafka/KafkaProducerService.java +++ b/queue/src/main/java/com/bazaarvoice/emodb/queue/core/kafka/KafkaProducerService.java @@ -28,13 +28,13 @@ public KafkaProducerService() { * @param topic The Kafka topic. * @param events The collection of messages to be sent. */ - public void sendMessages(String topic, Collection events, String queueType) { + public void sendMessages(String topic, Collection events, String queueType) { LocalDateTime startTime = LocalDateTime.now(); _log.info("Sending {} messages to topic '{}'", events.size(), topic); List> futures = new ArrayList<>(); // Use async sendMessage and collect futures - for (String event : events) { - futures.add(producer.send(new ProducerRecord<>(topic, event))); + for (T event : events) { + futures.add(producer.send(new ProducerRecord<>(topic, event.toString()))); } // Wait for all futures to complete @@ -46,7 +46,7 @@ public void sendMessages(String topic, Collection events, String queueTy throw new RuntimeException("Error sending messages to Kafka", e); } } - _log.info("Finished sending messages to topic '{}' time taken : {} milliseconds", topic, Duration.between(startTime,LocalDateTime.now()).toMillis()); + _log.info("Finished sending messages to topic '{}' time taken : {} milliseconds", topic, Duration.between(startTime, LocalDateTime.now()).toMillis()); } diff --git a/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/DataStore.java b/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/DataStore.java index 7a00eedf9f..711c615250 100644 --- a/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/DataStore.java +++ b/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/DataStore.java @@ -261,4 +261,10 @@ void dropFacade(String table, String placement, Audit audit) */ URI getStashRoot() throws StashNotAvailableException; + + default void updateRefInDatabus(Iterable updateRefs, Set tags, boolean isFacade) { + /* + * This method is a no-op in the default implementation. It is used by the Databus to update the reference + */ + } } diff --git a/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/Names.java b/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/Names.java index 34d020b1fe..e60327738a 100644 --- a/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/Names.java +++ b/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/Names.java @@ -1,10 +1,18 @@ package com.bazaarvoice.emodb.sor.api; +import com.google.common.base.CharMatcher; + public abstract class Names { /** Prevent instantiation. */ private Names() {} + private static final CharMatcher QUEUE_NAME_ALLOWED = + CharMatcher.inRange('a', 'z') + .or(CharMatcher.inRange('0', '9')) + .or(CharMatcher.anyOf("-.:@_")) + .precomputed(); + /** * Table names must be lowercase ASCII strings. between 1 and 255 characters in length. Whitespace, ISO control * characters and certain punctuation characters that aren't generally allowed in file names or in elasticsearch @@ -19,4 +27,18 @@ public static boolean isLegalTableAttributeName(String attributeName) { // The attributes should not start with "~" which is reserved for Emodb's internal use return !attributeName.startsWith("~"); } + + /** + * Queue names must be lowercase ASCII strings. between 1 and 255 characters in length. Whitespace, ISO control + * characters and most punctuation characters aren't allowed. Queue names may not begin with a single underscore + * to allow URL space for extensions such as "/_queue/...". Queue names may not look like relative paths, ie. + * "." or "..". + */ + public static boolean isLegalQueueName(String queue) { + return queue != null && + queue.length() > 0 && queue.length() <= 255 && + !(queue.charAt(0) == '_' && !queue.startsWith("__")) && + !(queue.charAt(0) == '.' && (".".equals(queue) || "..".equals(queue))) && + QUEUE_NAME_ALLOWED.matchesAllOf(queue); + } } diff --git a/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/UpdateRefModel.java b/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/UpdateRefModel.java new file mode 100644 index 0000000000..168132b98a --- /dev/null +++ b/sor-api/src/main/java/com/bazaarvoice/emodb/sor/api/UpdateRefModel.java @@ -0,0 +1,63 @@ +package com.bazaarvoice.emodb.sor.api; + +import java.util.Objects; +import java.util.Set; +import java.util.UUID; + +import static java.util.Objects.hash; +import static java.util.Objects.requireNonNull; + +/** + * Reference to a System of Record update. + */ +public final class UpdateRefModel { + public static final String TAGS_NAME = "~tags"; + private final String _table; + private final String _key; + private final UUID _changeId; + private final Set _tags; + + public UpdateRefModel(String table, String key, UUID changeId, Set tags) { + _table = requireNonNull(table, "table"); + _key = requireNonNull(key, "key"); + _changeId = requireNonNull(changeId, "changeId"); + _tags = requireNonNull(tags, "tags"); + } + + public String getTable() { + return _table; + } + + public String getKey() { + return _key; + } + + public UUID getChangeId() { + return _changeId; + } + + public Set getTags() { + return _tags; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof UpdateRefModel)) { + return false; + } + UpdateRefModel that = (UpdateRefModel) o; + return Objects.equals(_table, that._table) && + Objects.equals(_key, that._key) && + Objects.equals(_changeId, that._changeId) && + Objects.equals(_tags, that._tags); + } + + @Override + public int hashCode() { + return hash(_table, _key, _changeId, _tags); + } + +} \ No newline at end of file diff --git a/sor/pom.xml b/sor/pom.xml index 4adf86fdc3..e678cd1c6a 100644 --- a/sor/pom.xml +++ b/sor/pom.xml @@ -348,5 +348,11 @@ testng test + + com.bazaarvoice.emodb + emodb-queue + 6.5.205-SNAPSHOT + compile + diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java index dadbdd3b4e..e52f2bc030 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java @@ -5,29 +5,11 @@ import com.bazaarvoice.emodb.common.json.deferred.LazyJsonMap; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; import com.bazaarvoice.emodb.common.zookeeper.store.MapStore; -import com.bazaarvoice.emodb.sor.api.Audit; -import com.bazaarvoice.emodb.sor.api.AuditBuilder; -import com.bazaarvoice.emodb.sor.api.AuditsUnavailableException; -import com.bazaarvoice.emodb.sor.api.Change; -import com.bazaarvoice.emodb.sor.api.CompactionControlSource; -import com.bazaarvoice.emodb.sor.api.Coordinate; -import com.bazaarvoice.emodb.sor.api.DataStore; -import com.bazaarvoice.emodb.sor.api.DefaultTable; -import com.bazaarvoice.emodb.sor.api.FacadeOptions; -import com.bazaarvoice.emodb.sor.api.History; -import com.bazaarvoice.emodb.sor.api.Intrinsic; -import com.bazaarvoice.emodb.sor.api.Names; -import com.bazaarvoice.emodb.sor.api.ReadConsistency; -import com.bazaarvoice.emodb.sor.api.StashNotAvailableException; -import com.bazaarvoice.emodb.sor.api.StashRunTimeInfo; -import com.bazaarvoice.emodb.sor.api.StashTimeKey; -import com.bazaarvoice.emodb.sor.api.TableOptions; -import com.bazaarvoice.emodb.sor.api.UnknownPlacementException; -import com.bazaarvoice.emodb.sor.api.UnknownTableException; -import com.bazaarvoice.emodb.sor.api.UnpublishedDatabusEvent; -import com.bazaarvoice.emodb.sor.api.UnpublishedDatabusEventType; -import com.bazaarvoice.emodb.sor.api.Update; -import com.bazaarvoice.emodb.sor.api.WriteConsistency; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaConfig; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; +import com.bazaarvoice.emodb.queue.core.ssm.ParameterStoreUtil; +import com.bazaarvoice.emodb.sor.api.*; import com.bazaarvoice.emodb.sor.audit.AuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.LocalCompactionControl; import com.bazaarvoice.emodb.sor.condition.Condition; @@ -59,6 +41,8 @@ import com.google.common.base.Function; import com.google.common.base.Supplier; import com.google.common.base.Throwables; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterators; @@ -79,15 +63,7 @@ import java.time.Duration; import java.time.Instant; import java.time.temporal.ChronoUnit; -import java.util.Collection; -import java.util.Collections; -import java.util.Date; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.UUID; +import java.util.*; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.RejectedExecutionException; @@ -104,6 +80,10 @@ public class DefaultDataStore implements DataStore, DataProvider, DataTools, Tab private static final int NUM_COMPACTION_THREADS = 2; private static final int MAX_COMPACTION_QUEUE_LENGTH = 100; + private static final String SYSTEM_PREFIX = "__system_bus:"; + private static final String MASTER_FANOUT = SYSTEM_PREFIX + "master"; + private static final String UNIVERSE = KafkaConfig.getUniverseFromEnv(); + private static final String DATA_THROTTLER = "databusThrottler"; private final Logger _log = LoggerFactory.getLogger(DefaultDataStore.class); @@ -126,6 +106,12 @@ public class DefaultDataStore implements DataStore, DataProvider, DataTools, Tab private final CompactionControlSource _compactionControlSource; private final MapStore _minSplitSizeMap; private final Clock _clock; + private final KafkaProducerService _kafkaProducerService; + private ParameterStoreUtil parameterStoreUtil; + private final BaseEventStore _eventStore; + private final Cache dataThrottlerCache = CacheBuilder.newBuilder() + .expireAfterWrite(1, TimeUnit.MINUTES) + .build(); private StashTableDAO _stashTableDao; @@ -134,10 +120,10 @@ public DefaultDataStore(LifeCycleRegistry lifeCycle, MetricRegistry metricRegist DataReaderDAO dataReaderDao, DataWriterDAO dataWriterDao, SlowQueryLog slowQueryLog, HistoryStore historyStore, @StashRoot Optional stashRootDirectory, @LocalCompactionControl CompactionControlSource compactionControlSource, @StashBlackListTableCondition Condition stashBlackListTableCondition, AuditWriter auditWriter, - @MinSplitSizeMap MapStore minSplitSizeMap, Clock clock) { + @MinSplitSizeMap MapStore minSplitSizeMap, Clock clock, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { this(eventWriterRegistry, tableDao, dataReaderDao, dataWriterDao, slowQueryLog, defaultCompactionExecutor(lifeCycle), historyStore, stashRootDirectory, compactionControlSource, stashBlackListTableCondition, auditWriter, - minSplitSizeMap, metricRegistry, clock); + minSplitSizeMap, metricRegistry, clock, kafkaProducerService, eventStore); } @VisibleForTesting @@ -146,7 +132,7 @@ public DefaultDataStore(DatabusEventWriterRegistry eventWriterRegistry,TableDAO SlowQueryLog slowQueryLog, ExecutorService compactionExecutor, HistoryStore historyStore, Optional stashRootDirectory, CompactionControlSource compactionControlSource, Condition stashBlackListTableCondition, AuditWriter auditWriter, - MapStore minSplitSizeMap, MetricRegistry metricRegistry, Clock clock) { + MapStore minSplitSizeMap, MetricRegistry metricRegistry, Clock clock, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { _eventWriterRegistry = requireNonNull(eventWriterRegistry, "eventWriterRegistry"); _tableDao = requireNonNull(tableDao, "tableDao"); _dataReaderDao = requireNonNull(dataReaderDao, "dataReaderDao"); @@ -166,6 +152,9 @@ public DefaultDataStore(DatabusEventWriterRegistry eventWriterRegistry,TableDAO _compactionControlSource = requireNonNull(compactionControlSource, "compactionControlSource"); _minSplitSizeMap = requireNonNull(minSplitSizeMap, "minSplitSizeMap"); _clock = requireNonNull(clock, "clock"); + _kafkaProducerService = requireNonNull(kafkaProducerService, "kafkaProducerService"); + this.parameterStoreUtil = new ParameterStoreUtil(); + _eventStore = eventStore; } /** @@ -376,6 +365,34 @@ public AnnotatedContent apply(Record record) { }; } + /** + * Retrieves the value of the "DataThrottler" flag from the cache if available. + * If the value is not present in the cache or the cache has expired, it fetches the value + * from AWS Parameter Store and stores it in the cache. + *

+ * The cached value has a TTL (Time-To-Live) of 5 minutes, after which it will be refreshed + * from the Parameter Store on the next access. + *

+ * + * @return {@code true} if the experiment is still running, otherwise {@code false}. + * @throws RuntimeException if there is an error fetching the value from the cache or Parameter Store. + */ + private boolean getDataThrottlerValue() { + try { + // Attempt to retrieve from cache + return dataThrottlerCache.get(DATA_THROTTLER, () -> { + + Boolean checkDataThrottler = Boolean.parseBoolean(parameterStoreUtil.getParameter("/" + UNIVERSE + "/emodb/" + DATA_THROTTLER)); + _log.info("DATA_THROTTLER is refreshed {}", checkDataThrottler); + // If absent or expired, fetch from Parameter Store and cache the result + return checkDataThrottler; + }); + } catch (Exception e) { + _log.error("Error fetching databusThrottler valie{}", e.getMessage()); + return false; + } + } + /** * Resolve a set of changes read from the {@link DataWriterDAO} into a single JSON literal object + metadata. * If the record can be compacted an asynchronous compaction will be scheduled unless @@ -745,6 +762,10 @@ public void beforeWrite(Collection updateBatch) { } if (!updateRefs.isEmpty()) { _eventWriterRegistry.getDatabusWriter().writeEvents(updateRefs); + if(getDataThrottlerValue()) + _kafkaProducerService.sendMessages(MASTER_FANOUT, updateRefs, "update"); + else + _eventWriterRegistry.getDatabusWriter().writeEvents(updateRefs); } } @@ -1025,4 +1046,30 @@ private void decrementDeltaSizes(PendingCompaction pendingCompaction) { private String getMetricName(String name) { return MetricRegistry.name("bv.emodb.sor", "DefaultDataStore", name); } + + private Iterable convertToUpdateRef(Iterable apiUpdateRefs) { + List coreUpdateRefs = new ArrayList<>(); + for (UpdateRefModel apiUpdateRefModel : apiUpdateRefs) { + String tableName = apiUpdateRefModel.getTable(); + String key = apiUpdateRefModel.getKey(); + UUID changeId = apiUpdateRefModel.getChangeId(); + Set tags = apiUpdateRefModel.getTags(); + coreUpdateRefs.add(new com.bazaarvoice.emodb.sor.core.UpdateRef(tableName, key, changeId, tags)); + } + return coreUpdateRefs; + } + + @Override + public void updateRefInDatabus(Iterable updateRefs, Set tags, boolean isFacade) { + Iterator updateRefsIter = convertToUpdateRef(updateRefs).iterator(); + if (!updateRefsIter.hasNext()) { + return; + } + while (updateRefsIter.hasNext()) { + UpdateRef updateRef = updateRefsIter.next(); + List updateRefList = Lists.newArrayListWithCapacity(1); + updateRefList.add(updateRef); + _eventWriterRegistry.getDatabusWriter().writeEvents(updateRefList); + } + } } diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java index 37348e976b..ec6cd60330 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java @@ -1,5 +1,7 @@ package com.bazaarvoice.emodb.sor.core.test; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.audit.DiscardingAuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.InMemoryCompactionControlSource; import com.bazaarvoice.emodb.sor.condition.Conditions; @@ -19,18 +21,19 @@ */ public class InMemoryDataStore extends DefaultDataStore { - public InMemoryDataStore(MetricRegistry metricRegistry) { - this(new InMemoryDataReaderDAO(), metricRegistry); + public InMemoryDataStore(MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { + this(new InMemoryDataReaderDAO(), metricRegistry, kafkaProducerService, eventStore); } - public InMemoryDataStore(InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry) { - this(new DatabusEventWriterRegistry(), dataDao, metricRegistry); + + public InMemoryDataStore(InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { + this(new DatabusEventWriterRegistry(), dataDao, metricRegistry, kafkaProducerService, eventStore); } - public InMemoryDataStore(DatabusEventWriterRegistry eventWriterRegistry, InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry) { + public InMemoryDataStore(DatabusEventWriterRegistry eventWriterRegistry, InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { super(eventWriterRegistry, new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), MoreExecutors.newDirectExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC(), kafkaProducerService, eventStore); } } diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java index 48da779c69..0cb2483f24 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java @@ -1,5 +1,7 @@ package com.bazaarvoice.emodb.sor.core; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; import com.bazaarvoice.emodb.sor.api.ChangeBuilder; @@ -485,7 +487,7 @@ public void compact(Table table, String key, UUID compactionKey, Compaction comp } }; - final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry()); + final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); // Create a table for our test dataStore.createTable(tableName, @@ -571,7 +573,7 @@ public Record read(Key key, ReadConsistency ignored) { // Configure the data DAO to read 10 columns initially, causing other column reads to be read lazily dataDAO.setColumnBatchSize(10); - final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry()); + final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); // Create a table for our test dataStore.createTable(tableName, diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java index 3cf9b7b50f..6fabfcf1e4 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java @@ -1,5 +1,7 @@ package com.bazaarvoice.emodb.sor.core; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; @@ -33,6 +35,7 @@ import java.util.Set; import java.util.UUID; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotEquals; @@ -47,7 +50,7 @@ public class DataStoreTest { @Test public void testDeltas() throws Exception { - DataStore store = new InMemoryDataStore(new MetricRegistry()); + DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(),mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); assertFalse(store.getTableExists(TABLE)); @@ -167,7 +170,7 @@ public void testDeltas() throws Exception { @Test public void testRecordTimestamps() throws Exception { - DataStore store = new InMemoryDataStore(new MetricRegistry()); + DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(),mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); assertFalse(store.getTableExists(TABLE)); @@ -262,7 +265,7 @@ record = store.get(TABLE, KEY1); @Test public void testRecordTimestampsWithEventTags() throws Exception { - DataStore store = new InMemoryDataStore(new MetricRegistry()); + DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(),mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); assertFalse(store.getTableExists(TABLE)); diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java index ac585fa220..b32e194d91 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java @@ -1,5 +1,7 @@ package com.bazaarvoice.emodb.sor.core; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.TableOptionsBuilder; @@ -43,7 +45,7 @@ public List getSplits(Table table, int recordsPerSplit, int localResplit } }; - DataStore dataStore = new InMemoryDataStore(dataDao, new MetricRegistry()); + DataStore dataStore = new InMemoryDataStore(dataDao, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); dataStore.createTable("table", new TableOptionsBuilder().setPlacement("default").build(), Collections.emptyMap(), new AuditBuilder().build()); diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java index 7377838dc5..8c14f5e7ee 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java @@ -2,6 +2,8 @@ import com.bazaarvoice.emodb.common.json.JsonHelper; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; @@ -65,7 +67,7 @@ public void testRedundantDeltas() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -122,7 +124,7 @@ public void testMinUUIDDelta() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -159,7 +161,7 @@ public void testRedundancyWithTags() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -240,7 +242,7 @@ public void testTagsForNestedMapDeltas() { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -260,7 +262,7 @@ public void testRedundancyWithCompactionAndUnchangedTag() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -337,7 +339,7 @@ public void testPartialCompactionWithNoRedundancy() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), tableDao, dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -409,7 +411,7 @@ public void testPartialCompactionWithRedundancy() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), tableDao, dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java index 60574f680c..765ce00d1d 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java @@ -1,6 +1,8 @@ package com.bazaarvoice.emodb.sor.core; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.TableOptionsBuilder; @@ -19,6 +21,7 @@ import java.util.UUID; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.fail; @@ -34,7 +37,7 @@ public class SorUpdateTest { public void SetupTest() { final InMemoryDataReaderDAO dataDAO = new InMemoryDataReaderDAO(); _eventWriterRegistry = new DatabusEventWriterRegistry(); - _dataStore = new InMemoryDataStore(_eventWriterRegistry, dataDAO, new MetricRegistry()); + _dataStore = new InMemoryDataStore(_eventWriterRegistry, dataDAO, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); // Create a table for our test diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java index c67f985342..35f9d5c86c 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java @@ -1,6 +1,8 @@ package com.bazaarvoice.emodb.sor.test; import com.bazaarvoice.emodb.common.dropwizard.lifecycle.SimpleLifeCycleRegistry; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.audit.DiscardingAuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.InMemoryCompactionControlSource; @@ -20,6 +22,8 @@ import java.time.Clock; import java.util.Optional; +import static org.mockito.Mockito.mock; + /** * Wrapper around a set of {@link DataStore} instances that replicate to each other, * simulating a set of eventually consistent data centers. @@ -63,12 +67,12 @@ public MultiDCDataStores(int numDCs, boolean asyncCompacter, MetricRegistry metr if (asyncCompacter) { _stores[i] = new DefaultDataStore(new SimpleLifeCycleRegistry(), metricRegistry, new DatabusEventWriterRegistry(), _tableDao, _inMemoryDaos[i].setHistoryStore(_historyStores[i]), _replDaos[i], new NullSlowQueryLog(), _historyStores[i], - Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), new DiscardingAuditWriter(), new InMemoryMapStore<>(), Clock.systemUTC()); + Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), new DiscardingAuditWriter(), new InMemoryMapStore<>(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); } else { _stores[i] = new DefaultDataStore(new DatabusEventWriterRegistry(), _tableDao, _inMemoryDaos[i].setHistoryStore(_historyStores[i]), _replDaos[i], new NullSlowQueryLog(), MoreExecutors.newDirectExecutorService(), _historyStores[i], Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC()); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); } } } diff --git a/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java b/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java index 5ad5ff357f..bb6009e0f2 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java @@ -11,6 +11,8 @@ import com.bazaarvoice.emodb.common.zookeeper.store.ValueStore; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.core.DefaultDataCenter; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.FacadeExistsException; @@ -1981,7 +1983,7 @@ dataCenter, mock(RateLimiterCache.class), dataCopyDAO, dataPurgeDAO, } private InMemoryDataStore newBackingStore(MetricRegistry metricRegistry) { - InMemoryDataStore store = new InMemoryDataStore(metricRegistry); + InMemoryDataStore store = new InMemoryDataStore(metricRegistry, new KafkaProducerService(), mock(BaseEventStore.class)); store.createTable("__system:table", newOptions(PL_GLOBAL), ImmutableMap.of(), newAudit()); store.createTable("__system:table_uuid", newOptions(PL_GLOBAL), ImmutableMap.of(), newAudit()); store.createTable("__system:table_unpublished_databus_events", newOptions(PL_GLOBAL), ImmutableMap.of(), newAudit()); diff --git a/web/src/main/java/com/bazaarvoice/emodb/web/resources/sor/DataStoreResource1.java b/web/src/main/java/com/bazaarvoice/emodb/web/resources/sor/DataStoreResource1.java index 27baf89a04..2efd03733e 100644 --- a/web/src/main/java/com/bazaarvoice/emodb/web/resources/sor/DataStoreResource1.java +++ b/web/src/main/java/com/bazaarvoice/emodb/web/resources/sor/DataStoreResource1.java @@ -8,19 +8,7 @@ import com.bazaarvoice.emodb.common.json.OrderedJson; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; import com.bazaarvoice.emodb.datacenter.api.DataCenter; -import com.bazaarvoice.emodb.sor.api.Audit; -import com.bazaarvoice.emodb.sor.api.Change; -import com.bazaarvoice.emodb.sor.api.CompactionControlSource; -import com.bazaarvoice.emodb.sor.api.Coordinate; -import com.bazaarvoice.emodb.sor.api.DataStore; -import com.bazaarvoice.emodb.sor.api.FacadeOptions; -import com.bazaarvoice.emodb.sor.api.Intrinsic; -import com.bazaarvoice.emodb.sor.api.PurgeStatus; -import com.bazaarvoice.emodb.sor.api.Table; -import com.bazaarvoice.emodb.sor.api.TableOptions; -import com.bazaarvoice.emodb.sor.api.UnpublishedDatabusEvent; -import com.bazaarvoice.emodb.sor.api.Update; -import com.bazaarvoice.emodb.sor.api.WriteConsistency; +import com.bazaarvoice.emodb.sor.api.*; import com.bazaarvoice.emodb.sor.core.DataStoreAsync; import com.bazaarvoice.emodb.sor.delta.Delta; import com.bazaarvoice.emodb.sor.delta.Deltas; @@ -947,6 +935,59 @@ private Response redirectTo(DataCenter dataCenter, URI requestUri) { build(); } + @POST + @Path("{channel}/sendbatch1") + @Consumes(MediaType.APPLICATION_JSON) + @Timed(name = "bv.emodb.sor.DataStoreResource1.updateRef", absolute = true) + @ApiOperation(value = "Updates a reference", + notes = "Updates a reference", + response = SuccessResponse.class + ) + public SuccessResponse updateRefToDatabus(InputStream in, + @QueryParam("consistency") @DefaultValue("STRONG") WriteConsistencyParam consistency, + @QueryParam("tag") List tags, + @Authenticated Subject subject) { + Set tagsSet = (tags == null) ? ImmutableSet.of() : Sets.newHashSet(tags); + Iterable updateRefs = asSubjectSafeUpdateRefModelIterable(new JsonStreamingArrayParser<>(in, UpdateRefModel.class), subject, true); + // Perform the update by writing to Databus + _dataStore.updateRefInDatabus(updateRefs, tagsSet, false); + return SuccessResponse.instance(); + } + + /** + * Takes an update ref stream from a subject and performs the following actions on it: + * 1. Checks that the subject has permission to update the record being updated + * 2. Applies any active rate limiting for updates by the subject + */ + private Iterable asSubjectSafeUpdateRefModelIterable(Iterator updateRefs, final Subject subject, final boolean isFacade) { + return Iterables.filter( + OneTimeIterable.wrap(updateRefs), + new Predicate() { + @Override + public boolean apply(UpdateRefModel updateRefModel) { + NamedResource resource = new NamedResource(updateRefModel.getTable()); + boolean hasPermission; + if (isFacade) { + hasPermission = subject.hasPermission(Permissions.updateFacade(resource)); + } else { + hasPermission = subject.hasPermission(Permissions.updateSorTable(resource)); + } + + if (!hasPermission) { + throw new UnauthorizedException("not authorized to update table " + updateRefModel.getTable()); + } + + // Facades are a unique case used internally for shoveling data across data centers, so don't rate + // limit facade updates. + if (!isFacade) { + _updateThrottle.beforeUpdate(subject.getId()); + } + + return true; + } + }); + } + private UUID parseUuidOrTimestamp(@Nullable String string, boolean rangeUpperEnd) { if (string == null) { return null; diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java index fd11acab47..4bdc43f9d5 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java @@ -2,6 +2,7 @@ import com.bazaarvoice.emodb.common.dropwizard.lifecycle.LifeCycleRegistry; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; +import com.bazaarvoice.emodb.event.api.BaseEventStore; import com.bazaarvoice.emodb.job.api.JobIdentifier; import com.bazaarvoice.emodb.job.api.JobRequest; import com.bazaarvoice.emodb.job.api.JobStatus; @@ -11,6 +12,7 @@ import com.bazaarvoice.emodb.job.handler.DefaultJobHandlerRegistry; import com.bazaarvoice.emodb.job.service.DefaultJobService; import com.bazaarvoice.emodb.queue.api.QueueService; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.CompactionControlSource; @@ -84,7 +86,7 @@ public void setUp() throws Exception { lifeCycleRegistry, _queueService, "testqueue", _jobHandlerRegistry, _jobStatusDAO, _curator, 1, Duration.ZERO, 100, Duration.ofHours(1)); - _store = new InMemoryDataStore(new MetricRegistry()); + _store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); _dataStoreResource = new DataStoreResource1(_store, new DefaultDataStoreAsync(_store, _service, _jobHandlerRegistry), mock(CompactionControlSource.class), new UnlimitedDataStoreUpdateThrottler()); diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java index a96563df8d..d5a4eaa125 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java @@ -12,9 +12,11 @@ import com.bazaarvoice.emodb.common.json.JsonHelper; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.api.DataCenters; +import com.bazaarvoice.emodb.event.api.BaseEventStore; import com.bazaarvoice.emodb.plugin.stash.StashMetadata; import com.bazaarvoice.emodb.plugin.stash.StashStateListener; import com.bazaarvoice.emodb.queue.core.ByteBufferInputStream; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.CompactionControlSource; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.api.ReadConsistency; @@ -420,8 +422,9 @@ dataTools, scanWriterGenerator, compactionControlSource, mock(LifeCycleRegistry. @Test public void testScanUploadFromExistingScan() throws Exception { MetricRegistry metricRegistry = new MetricRegistry(); + KafkaProducerService kafkaProducerService = new KafkaProducerService(); // Use an in-memory data store but override the default splits operation to return 4 splits for the test placement - InMemoryDataStore dataStore = spy(new InMemoryDataStore(metricRegistry)); + InMemoryDataStore dataStore = spy(new InMemoryDataStore(metricRegistry, kafkaProducerService, mock(BaseEventStore.class))); when(dataStore.getScanRangeSplits("app_global:default", 1000000, Optional.empty())) .thenReturn(new ScanRangeSplits(ImmutableList.of( createSimpleSplitGroup("00", "40"), @@ -621,7 +624,7 @@ public void testScanFailureRecovery() Lists.newArrayList(), Lists.newArrayList()); InMemoryScanWorkflow scanWorkflow = new InMemoryScanWorkflow(); - ScanStatusDAO scanStatusDAO = new DataStoreScanStatusDAO(new InMemoryDataStore(new MetricRegistry()), "scan_table", "app_global:sys"); + ScanStatusDAO scanStatusDAO = new DataStoreScanStatusDAO(new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)), "scan_table", "app_global:sys"); LocalScanUploadMonitor monitor = new LocalScanUploadMonitor(scanWorkflow, scanStatusDAO, mock(ScanWriterGenerator.class), mock(StashStateListener.class), mock(ScanCountListener.class), mock(DataTools.class), new InMemoryCompactionControlSource(), mock(DataCenters.class)); diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java index 0c19fa2c14..2581db11fd 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java @@ -1,6 +1,8 @@ package com.bazaarvoice.emodb.web.scanner.scanstatus; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; @@ -22,6 +24,7 @@ import java.util.List; import java.util.Optional; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; @@ -33,7 +36,7 @@ public class DataStoreScanStatusDAOTest { @BeforeMethod public void setUp() { - _dataStore = new InMemoryDataStore(new MetricRegistry()); + _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); _dao = new DataStoreScanStatusDAO(_dataStore, "scan_table", "app_global:sys"); } diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java index 4e14f50fd1..a5df6effb3 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java @@ -1,5 +1,7 @@ package com.bazaarvoice.emodb.web.scanner.scanstatus; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.codahale.metrics.MetricRegistry; @@ -9,6 +11,7 @@ import java.util.Date; +import static org.mockito.Mockito.mock; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; @@ -19,7 +22,7 @@ public class DataStoreStashRequestDAOTest { @BeforeMethod public void setUp() { - _dataStore = new InMemoryDataStore(new MetricRegistry()); + _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); _dao = new DataStoreStashRequestDAO(_dataStore, "request_table", "app_global:sys"); } diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java index f8c5758a07..1ad088d5ee 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java @@ -3,6 +3,8 @@ import com.bazaarvoice.emodb.cachemgr.api.CacheHandle; import com.bazaarvoice.emodb.cachemgr.api.CacheRegistry; import com.bazaarvoice.emodb.cachemgr.api.InvalidationScope; +import com.bazaarvoice.emodb.event.api.BaseEventStore; +import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; @@ -32,7 +34,7 @@ public class SettingsManagerTest { @BeforeMethod public void setUp() { - _dataStore = new InMemoryDataStore(new MetricRegistry()); + _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); _cacheRegistry = mock(CacheRegistry.class); _cacheHandle = mock(CacheHandle.class); when(_cacheRegistry.register(eq("settings"), any(Cache.class), eq(true))).thenReturn(_cacheHandle); From 0dad3e9296e12c9f4a9dd9d8b356baa508ab6733 Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Tue, 5 Nov 2024 09:28:50 +0000 Subject: [PATCH 2/7] duplicated kafka and ssm util packages in emo sor module --- .../auth/TableAuthIdentityManagerDAOTest.java | 9 +- .../auth/TableRoleManagerDAOTest.java | 5 +- .../integration/sor/CasStashTableTest.java | 5 +- sor/pom.xml | 10 +- .../emodb/sor/core/DefaultDataStore.java | 15 +- .../sor/core/test/InMemoryDataStore.java | 15 +- .../emodb/sor/kafka/KafkaAdminService.java | 118 +++++++++++++ .../emodb/sor/kafka/KafkaConfig.java | 157 ++++++++++++++++++ .../emodb/sor/kafka/KafkaProducerService.java | 66 ++++++++ .../emodb/sor/ssm/ParameterStoreUtil.java | 120 +++++++++++++ .../emodb/sor/core/CompactorTest.java | 7 +- .../emodb/sor/core/DataStoreTest.java | 9 +- .../emodb/sor/core/MinSplitSizeTest.java | 5 +- .../emodb/sor/core/RedundantDeltaTest.java | 17 +- .../emodb/sor/core/SorUpdateTest.java | 5 +- .../emodb/sor/test/MultiDCDataStores.java | 7 +- .../table/db/astyanax/TableLifeCycleTest.java | 5 +- .../emodb/web/purge/PurgeTest.java | 5 +- .../emodb/web/scanner/ScanUploaderTest.java | 7 +- .../DataStoreScanStatusDAOTest.java | 5 +- .../DataStoreStashRequestDAOTest.java | 5 +- .../web/settings/SettingsManagerTest.java | 5 +- 22 files changed, 523 insertions(+), 79 deletions(-) create mode 100644 sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java create mode 100644 sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java create mode 100644 sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java create mode 100644 sor/src/main/java/com/bazaarvoice/emodb/sor/ssm/ParameterStoreUtil.java diff --git a/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java b/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java index 67c3849a76..e6da43c029 100644 --- a/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java +++ b/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java @@ -3,13 +3,12 @@ import com.bazaarvoice.emodb.auth.apikey.ApiKey; import com.bazaarvoice.emodb.auth.apikey.ApiKeyModification; import com.bazaarvoice.emodb.auth.identity.TableAuthIdentityManagerDAO; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.uuid.TimeUUIDs; import com.codahale.metrics.MetricRegistry; import com.google.common.collect.ImmutableList; @@ -41,7 +40,7 @@ public class TableAuthIdentityManagerDAOTest { */ @Test public void testRebuildIdIndex() { - DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); Supplier idSupplier = () -> "id0"; TableAuthIdentityManagerDAO tableAuthIdentityManagerDAO = new TableAuthIdentityManagerDAO<>( ApiKey.class, dataStore, "__auth:keys", "__auth:internal_ids", "app_global:sys", @@ -79,7 +78,7 @@ public void testRebuildIdIndex() { @Test public void testGrandfatheredInId() { - DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); Supplier idSupplier = () -> "id0"; TableAuthIdentityManagerDAO tableAuthIdentityManagerDAO = new TableAuthIdentityManagerDAO<>( ApiKey.class, dataStore, "__auth:keys", "__auth:internal_ids", "app_global:sys", @@ -131,7 +130,7 @@ public void testGrandfatheredInId() { @Test public void testIdAttributeCompatibility() { - DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + DataStore dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); Supplier idSupplier = () -> "id0"; TableAuthIdentityManagerDAO tableAuthIdentityManagerDAO = new TableAuthIdentityManagerDAO<>( ApiKey.class, dataStore, "__auth:keys", "__auth:internal_ids", "app_global:sys", diff --git a/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java b/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java index 0c54beb6b4..a3db269d93 100644 --- a/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java +++ b/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java @@ -9,8 +9,6 @@ import com.bazaarvoice.emodb.auth.role.RoleModification; import com.bazaarvoice.emodb.auth.role.RoleNotFoundException; import com.bazaarvoice.emodb.auth.role.TableRoleManagerDAO; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; @@ -20,6 +18,7 @@ import com.bazaarvoice.emodb.sor.api.WriteConsistency; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.web.auth.EmoPermissionResolver; import com.codahale.metrics.MetricRegistry; import com.google.common.collect.ImmutableList; @@ -62,7 +61,7 @@ public class TableRoleManagerDAOTest { @BeforeMethod public void setUp() { // DataStore and PermissionManager are fairly heavy to fully mock. Use spies on in-memory implementations instead - _backendDataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _backendDataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); _dataStore = spy(_backendDataStore); _permissionResolver = new EmoPermissionResolver(null, null); _backendPermissionManager = new InMemoryPermissionManager(_permissionResolver); diff --git a/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java b/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java index 702cf97d40..79ce74873f 100644 --- a/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java +++ b/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java @@ -10,8 +10,6 @@ import com.bazaarvoice.emodb.common.zookeeper.store.ValueStore; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.api.DataCenters; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.condition.Conditions; @@ -19,6 +17,7 @@ import com.bazaarvoice.emodb.sor.db.astyanax.DeltaPlacementFactory; import com.bazaarvoice.emodb.sor.delta.Delta; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.table.db.Table; import com.bazaarvoice.emodb.table.db.astyanax.AstyanaxTableDAO; import com.bazaarvoice.emodb.table.db.astyanax.CQLStashTableDAO; @@ -99,7 +98,7 @@ public void setup() throws Exception { _astyanaxTableDAO.setCQLStashTableDAO(cqlStashTableDAO); // Don't store table definitions in the actual backing store so as not to interrupt other tests. Use a // private in-memory implementation. - _tableBackingStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _tableBackingStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); _astyanaxTableDAO.setBackingStore(_tableBackingStore); _lifeCycleRegistry.start(); diff --git a/sor/pom.xml b/sor/pom.xml index e678cd1c6a..a93fb4b0b5 100644 --- a/sor/pom.xml +++ b/sor/pom.xml @@ -349,10 +349,12 @@ test - com.bazaarvoice.emodb - emodb-queue - 6.5.205-SNAPSHOT - compile + com.amazonaws + aws-java-sdk-ssm + + + org.apache.kafka + kafka-clients diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java index e52f2bc030..4b6475b598 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java @@ -5,10 +5,6 @@ import com.bazaarvoice.emodb.common.json.deferred.LazyJsonMap; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; import com.bazaarvoice.emodb.common.zookeeper.store.MapStore; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaConfig; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; -import com.bazaarvoice.emodb.queue.core.ssm.ParameterStoreUtil; import com.bazaarvoice.emodb.sor.api.*; import com.bazaarvoice.emodb.sor.audit.AuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.LocalCompactionControl; @@ -24,7 +20,10 @@ import com.bazaarvoice.emodb.sor.db.ScanRange; import com.bazaarvoice.emodb.sor.db.ScanRangeSplits; import com.bazaarvoice.emodb.sor.delta.Delta; +import com.bazaarvoice.emodb.sor.kafka.KafkaConfig; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.log.SlowQueryLog; +import com.bazaarvoice.emodb.sor.ssm.ParameterStoreUtil; import com.bazaarvoice.emodb.table.db.DroppedTableException; import com.bazaarvoice.emodb.table.db.StashBlackListTableCondition; import com.bazaarvoice.emodb.table.db.StashTableDAO; @@ -108,7 +107,6 @@ public class DefaultDataStore implements DataStore, DataProvider, DataTools, Tab private final Clock _clock; private final KafkaProducerService _kafkaProducerService; private ParameterStoreUtil parameterStoreUtil; - private final BaseEventStore _eventStore; private final Cache dataThrottlerCache = CacheBuilder.newBuilder() .expireAfterWrite(1, TimeUnit.MINUTES) .build(); @@ -120,10 +118,10 @@ public DefaultDataStore(LifeCycleRegistry lifeCycle, MetricRegistry metricRegist DataReaderDAO dataReaderDao, DataWriterDAO dataWriterDao, SlowQueryLog slowQueryLog, HistoryStore historyStore, @StashRoot Optional stashRootDirectory, @LocalCompactionControl CompactionControlSource compactionControlSource, @StashBlackListTableCondition Condition stashBlackListTableCondition, AuditWriter auditWriter, - @MinSplitSizeMap MapStore minSplitSizeMap, Clock clock, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { + @MinSplitSizeMap MapStore minSplitSizeMap, Clock clock, KafkaProducerService kafkaProducerService) { this(eventWriterRegistry, tableDao, dataReaderDao, dataWriterDao, slowQueryLog, defaultCompactionExecutor(lifeCycle), historyStore, stashRootDirectory, compactionControlSource, stashBlackListTableCondition, auditWriter, - minSplitSizeMap, metricRegistry, clock, kafkaProducerService, eventStore); + minSplitSizeMap, metricRegistry, clock, kafkaProducerService); } @VisibleForTesting @@ -132,7 +130,7 @@ public DefaultDataStore(DatabusEventWriterRegistry eventWriterRegistry,TableDAO SlowQueryLog slowQueryLog, ExecutorService compactionExecutor, HistoryStore historyStore, Optional stashRootDirectory, CompactionControlSource compactionControlSource, Condition stashBlackListTableCondition, AuditWriter auditWriter, - MapStore minSplitSizeMap, MetricRegistry metricRegistry, Clock clock, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { + MapStore minSplitSizeMap, MetricRegistry metricRegistry, Clock clock, KafkaProducerService kafkaProducerService) { _eventWriterRegistry = requireNonNull(eventWriterRegistry, "eventWriterRegistry"); _tableDao = requireNonNull(tableDao, "tableDao"); _dataReaderDao = requireNonNull(dataReaderDao, "dataReaderDao"); @@ -154,7 +152,6 @@ public DefaultDataStore(DatabusEventWriterRegistry eventWriterRegistry,TableDAO _clock = requireNonNull(clock, "clock"); _kafkaProducerService = requireNonNull(kafkaProducerService, "kafkaProducerService"); this.parameterStoreUtil = new ParameterStoreUtil(); - _eventStore = eventStore; } /** diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java index ec6cd60330..8f6d9da161 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java @@ -1,13 +1,12 @@ package com.bazaarvoice.emodb.sor.core.test; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.audit.DiscardingAuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.InMemoryCompactionControlSource; import com.bazaarvoice.emodb.sor.condition.Conditions; import com.bazaarvoice.emodb.sor.core.DatabusEventWriterRegistry; import com.bazaarvoice.emodb.sor.core.DefaultDataStore; import com.bazaarvoice.emodb.sor.db.test.InMemoryDataReaderDAO; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.log.NullSlowQueryLog; import com.bazaarvoice.emodb.table.db.test.InMemoryTableDAO; import com.codahale.metrics.MetricRegistry; @@ -21,19 +20,19 @@ */ public class InMemoryDataStore extends DefaultDataStore { - public InMemoryDataStore(MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { - this(new InMemoryDataReaderDAO(), metricRegistry, kafkaProducerService, eventStore); + public InMemoryDataStore(MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService) { + this(new InMemoryDataReaderDAO(), metricRegistry, kafkaProducerService); } - public InMemoryDataStore(InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { - this(new DatabusEventWriterRegistry(), dataDao, metricRegistry, kafkaProducerService, eventStore); + public InMemoryDataStore(InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService) { + this(new DatabusEventWriterRegistry(), dataDao, metricRegistry, kafkaProducerService); } - public InMemoryDataStore(DatabusEventWriterRegistry eventWriterRegistry, InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService, BaseEventStore eventStore) { + public InMemoryDataStore(DatabusEventWriterRegistry eventWriterRegistry, InMemoryDataReaderDAO dataDao, MetricRegistry metricRegistry, KafkaProducerService kafkaProducerService) { super(eventWriterRegistry, new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), MoreExecutors.newDirectExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC(), kafkaProducerService, eventStore); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC(), kafkaProducerService); } } diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java new file mode 100644 index 0000000000..8bdccc2918 --- /dev/null +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java @@ -0,0 +1,118 @@ +package com.bazaarvoice.emodb.sor.kafka; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +public class KafkaAdminService { + private static final Logger _log = LoggerFactory.getLogger(KafkaAdminService.class); + private final AdminClient adminClient; + // Cache for the list of all topics with a TTL of 10 minutes + private final Cache> topicListCache = CacheBuilder.newBuilder() + .expireAfterWrite(1, TimeUnit.MINUTES) + .build(); + + private static final String TOPIC_LIST_KEY = "allTopics"; + + + public KafkaAdminService() { + this.adminClient = AdminClient.create(KafkaConfig.getAdminProps()); + } + + /** + * Creates a new Kafka topic with the specified configurations. + * + * @param topic The name of the topic. + * @param numPartitions Number of partitions. + * @param replicationFactor Replication factor. + */ + public Boolean createTopicIfNotExists(String topic, int numPartitions, short replicationFactor, String queueType) { + Boolean isExisting =isTopicExists(topic); + if (! isExisting) { + //create the topic now + NewTopic newTopic = new NewTopic(topic, numPartitions, replicationFactor); + try { + adminClient.createTopics(Collections.singleton(newTopic)).all().get(); + addToCache(topic); + _log.info("Created topic: {} with numPartitions: {} and replication factor {} ", topic, numPartitions, replicationFactor); + } catch (Exception e) { + _log.error("Error creating topic {}: ", topic, e); + throw new RuntimeException(e); + } + } + return isExisting; + } + public void addToCache(String topic){ + Set topics = topicListCache.getIfPresent(TOPIC_LIST_KEY); + if (topics == null) { + topics = new HashSet<>(); + } else { + // Create a new mutable Set if the existing one is unmodifiable + topics = new HashSet<>(topics); + } + topics.add(topic); + topicListCache.put(TOPIC_LIST_KEY, topics); + _log.info("Added newly created topic to cache: {}", topic); + } + + + /** + * Checks if a Kafka topic exists by using a cache to store the list of all topics. + * If the cache entry has expired or the cache is empty, it queries the Kafka AdminClient for the topic list. + *

+ * The cached list has a TTL (Time-To-Live) of 10 minutes, after which it will be refreshed + * from Kafka on the next access. + *

+ * + * @param topic the name of the Kafka topic to check + * @return {@code true} if the topic exists, otherwise {@code false}. + * @throws RuntimeException if there is an error fetching the topic list or checking if the topic exists. + */ + public boolean isTopicExists(String topic) { + try { + // Retrieve the list of topics from the cache + Set topics = topicListCache.get(TOPIC_LIST_KEY, this::fetchTopicListFromKafka); + + // Check if the given topic is in the cached list + return topics.contains(topic); + } catch (ExecutionException e) { + _log.error("Failed to check if topic exists: {}", topic, e); + throw new RuntimeException("Error checking if topic exists", e); + } + } + + /** + * Fetches the list of all topic names from Kafka AdminClient. + * This method is called only when the cache is expired or empty. + * + * @return a Set containing all topic names. + * @throws ExecutionException if there is an error fetching the topic list from Kafka. + */ + private Set fetchTopicListFromKafka() throws ExecutionException { + try { + _log.info("Fetching topic list from Kafka"); + return adminClient.listTopics().names().get(); + } catch (Exception e) { + _log.error("Error fetching topic list from Kafka", e); + throw new ExecutionException(e); + } + } + + /** + * Closes the AdminClient to release resources. + */ + public void close() { + if (adminClient != null) { + adminClient.close(); + } + } +} \ No newline at end of file diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java new file mode 100644 index 0000000000..255bc53fb8 --- /dev/null +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java @@ -0,0 +1,157 @@ +package com.bazaarvoice.emodb.sor.kafka; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.simplesystemsmanagement.AWSSimpleSystemsManagement; +import com.amazonaws.services.simplesystemsmanagement.AWSSimpleSystemsManagementClientBuilder; +import com.amazonaws.services.simplesystemsmanagement.model.AWSSimpleSystemsManagementException; +import com.amazonaws.services.simplesystemsmanagement.model.GetParametersRequest; +import com.amazonaws.services.simplesystemsmanagement.model.GetParametersResult; +import com.amazonaws.services.simplesystemsmanagement.model.Parameter; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.StringSerializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedReader; +import java.io.FileReader; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +public class KafkaConfig { + private static String bootstrapServersConfig; + private static String batchSizeConfig; + private static String retriesConfig; + private static String lingerMsConfig; + private static final Logger logger = LoggerFactory.getLogger(KafkaConfig.class); + // Static SSM Client and configuration using AWS SDK v1 + private static final AWSSimpleSystemsManagement ssmClient = AWSSimpleSystemsManagementClientBuilder + .standard() + .build(); + + + static { + try { + final String UNIVERSE = getUniverseFromEnv(); + // Load configurations from SSM during static initialization + Map parameterValues = getParameterValues( + Arrays.asList( + "/" + UNIVERSE + "/emodb/kafka/batchSize", + "/" + UNIVERSE + "/emodb/kafka/retries", + "/" + UNIVERSE + "/emodb/kafka/lingerMs", + "/" + UNIVERSE + "/emodb/kafka/bootstrapServers" + ) + ); + + // Set configurations with fallback to defaults if not present + // Sets the batch size for Kafka producer, which controls the amount of data to batch before sending. + batchSizeConfig = parameterValues.getOrDefault("/" + UNIVERSE + "/emodb/kafka/batchSize", "16384"); + + // Sets the number of retry attempts for failed Kafka message sends. + retriesConfig = parameterValues.getOrDefault("/" + UNIVERSE + "/emodb/kafka/retries", "3"); + + // Sets the number of milliseconds a producer is willing to wait before sending a batch out + lingerMsConfig = parameterValues.getOrDefault("/" + UNIVERSE + "/emodb/kafka/lingerMs", "1"); + + // Configures the Kafka broker addresses for producer connections. + bootstrapServersConfig = parameterValues.get("/" + UNIVERSE + "/emodb/kafka/bootstrapServers"); + + logger.info("Kafka configurations loaded successfully from SSM."); + } catch (AmazonServiceException e) { + logger.error("Failed to load configurations from SSM. Using default values.", e); + throw e; + } + catch (Exception e) { + logger.error("Unexpected error occurred while loading configurations from SSM. Using default values.", e); + throw e; + } + } + + public static String getUniverseFromEnv() { + String filePath = "/etc/environment"; + logger.info("Reading environment file: " + filePath); + Properties environmentProps = new Properties(); + + try (BufferedReader reader = new BufferedReader(new FileReader(filePath))) { + String line; + while ((line = reader.readLine()) != null) { + // Skip empty lines or comments + if (line.trim().isEmpty() || line.trim().startsWith("#")) { + continue; + } + // Split the line into key-value pair + String[] parts = line.split("=", 2); + logger.info("parts: " + Arrays.toString(parts)); + if (parts.length == 2) { + String key = parts[0].trim(); + String value = parts[1].trim(); + // Remove any surrounding quotes from value + value = value.replace("\"", ""); + environmentProps.put(key, value); + } + } + // Access the environment variables + return environmentProps.getProperty("UNIVERSE"); + } catch (IOException e) { + logger.error("Error reading environment file: " + e.getMessage()); + throw new RuntimeException("Error reading environment file: " + e.getMessage()); + } + } + // Fetch parameters from AWS SSM using AWS SDK v1 + private static Map getParameterValues(List parameterNames) { + try { + GetParametersRequest request = new GetParametersRequest() + .withNames(parameterNames) + .withWithDecryption(true); + + GetParametersResult response = ssmClient.getParameters(request); + + return response.getParameters().stream() + .collect(Collectors.toMap(Parameter::getName, Parameter::getValue)); + } catch (AWSSimpleSystemsManagementException e) { + logger.error("Error fetching parameters from SSM.", e); + throw e; // Rethrow or handle the exception if necessary + } + } + + // Kafka Producer properties + public static Properties getProducerProps() { + Properties producerProps = new Properties(); + + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersConfig); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + producerProps.put(ProducerConfig.RETRIES_CONFIG, Integer.parseInt(retriesConfig)); + producerProps.put(ProducerConfig.LINGER_MS_CONFIG, Integer.parseInt(lingerMsConfig)); + producerProps.put(ProducerConfig.BATCH_SIZE_CONFIG, Integer.parseInt(batchSizeConfig)); + producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, 33554432); // Default buffer memory setting + logger.info("Kafka Producer properties initialized."); + return producerProps; + } + + // Kafka Admin properties + public static Properties getAdminProps() { + Properties adminProps = new Properties(); + + adminProps.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersConfig); + logger.info("Kafka Admin properties initialized."); + return adminProps; + } + + // Ensure the SSM client is closed when the application shuts down + public static void shutdown() { + if (ssmClient != null) { + try { + ssmClient.shutdown(); + logger.info("SSM client closed successfully."); + } catch (Exception e) { + logger.error("Error while closing SSM client.", e); + } + } + } +} \ No newline at end of file diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java new file mode 100644 index 0000000000..4cb587cf29 --- /dev/null +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java @@ -0,0 +1,66 @@ +package com.bazaarvoice.emodb.sor.kafka; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Future; + +public class KafkaProducerService { + private static final Logger _log = LoggerFactory.getLogger(KafkaProducerService.class); + private final KafkaProducer producer; // Changed to String + + public KafkaProducerService() { + this.producer = new KafkaProducer<>(KafkaConfig.getProducerProps()); + _log.info("KafkaProducerService initialized with producer properties: {}", KafkaConfig.getProducerProps()); + } + + /** + * Sends each message from the collection to the specified Kafka topic separately. + * + * @param topic The Kafka topic. + * @param events The collection of messages to be sent. + */ + public void sendMessages(String topic, Collection events, String queueType) { + LocalDateTime startTime = LocalDateTime.now(); + _log.info("Sending {} messages to topic '{}'", events.size(), topic); + List> futures = new ArrayList<>(); + // Use async sendMessage and collect futures + for (T event : events) { + futures.add(producer.send(new ProducerRecord<>(topic, event.toString()))); + } + + // Wait for all futures to complete + for (Future future : futures) { + try { + future.get(); // Only blocks if a future is not yet complete + } catch (Exception e) { + _log.error("Error while sending message to Kafka: {}", e.getMessage()); + throw new RuntimeException("Error sending messages to Kafka", e); + } + } + _log.info("Finished sending messages to topic '{}' time taken : {} milliseconds", topic, Duration.between(startTime, LocalDateTime.now()).toMillis()); + } + + + /** + * Closes the producer to release resources. + */ + public void close() { + _log.info("Closing Kafka producer."); + try { + producer.flush(); + producer.close(); + } catch (Exception e) { + _log.error("Error while closing Kafka producer: ", e); + throw e; + } + } +} \ No newline at end of file diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/ssm/ParameterStoreUtil.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/ssm/ParameterStoreUtil.java new file mode 100644 index 0000000000..693901b264 --- /dev/null +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/ssm/ParameterStoreUtil.java @@ -0,0 +1,120 @@ +package com.bazaarvoice.emodb.sor.ssm; + +import com.amazonaws.services.simplesystemsmanagement.AWSSimpleSystemsManagement; +import com.amazonaws.services.simplesystemsmanagement.AWSSimpleSystemsManagementClientBuilder; +import com.amazonaws.services.simplesystemsmanagement.model.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Utility class for interacting with AWS Parameter Store using AWS SDK v1. + */ +public class ParameterStoreUtil { + + private static final Logger logger = LoggerFactory.getLogger(ParameterStoreUtil.class); + private final AWSSimpleSystemsManagement ssmClient; + + /** + * Constructor to initialize the SSM client + */ + public ParameterStoreUtil() { + // Create SSM client with default credentials and region + ssmClient = AWSSimpleSystemsManagementClientBuilder.standard() + .build(); + } + + /** + * Fetches a parameter from AWS Parameter Store. + * + * @param parameterName The name of the parameter to fetch + * @return The value of the parameter + * @throws IllegalArgumentException If the parameterName is null or empty + */ + public String getParameter(String parameterName) { + if (parameterName == null || parameterName.isEmpty()) { + logger.error("Parameter name cannot be null or empty"); + throw new IllegalArgumentException("Parameter name cannot be null or empty"); + } + + try { + + GetParameterRequest request = new GetParameterRequest().withName(parameterName); + GetParameterResult result = ssmClient.getParameter(request); + return result.getParameter().getValue(); + + } catch (ParameterNotFoundException e) { + logger.error("Parameter not found: {}", parameterName, e); + throw new RuntimeException("Parameter not found: " + parameterName, e); + + } catch (AWSSimpleSystemsManagementException e) { + logger.error("Error fetching parameter from AWS SSM: {}", e.getMessage(), e); + throw new RuntimeException("Error fetching parameter from AWS SSM: " + parameterName, e); + + } catch (Exception e) { + logger.error("Unexpected error while fetching parameter: {}", parameterName, e); + throw new RuntimeException("Unexpected error fetching parameter: " + parameterName, e); + } + } + + /** + * Fetches multiple parameters from AWS Parameter Store in a batch. + * + * @param parameterNames The list of parameter names to fetch + * @return A map of parameter names to their values + * @throws IllegalArgumentException If the parameterNames list is null or empty + */ + public Map getParameters(List parameterNames) { + if (parameterNames == null || parameterNames.isEmpty()) { + logger.error("Parameter names list cannot be null or empty"); + throw new IllegalArgumentException("Parameter names list cannot be null or empty"); + } + + try { + + GetParametersRequest request = new GetParametersRequest().withNames(parameterNames); + GetParametersResult result = ssmClient.getParameters(request); + + // Map the result to a Map of parameter names and values + Map parameters = new HashMap<>(); + result.getParameters().forEach(param -> parameters.put(param.getName(), param.getValue())); + + // Log any parameters that were not found + if (!result.getInvalidParameters().isEmpty()) { + logger.warn("The following parameters were not found: {}", result.getInvalidParameters()); + } + + return parameters; + + } catch (AWSSimpleSystemsManagementException e) { + logger.error("Error fetching parameters from AWS SSM: {}", e.getMessage(), e); + throw new RuntimeException("Error fetching parameters from AWS SSM: " + parameterNames, e); + + } catch (Exception e) { + logger.error("Unexpected error while fetching parameters: {}", parameterNames, e); + throw new RuntimeException("Unexpected error fetching parameters: " + parameterNames, e); + } + } + + public Long updateParameter(String key, String value) { + try { + if (key == null || key.trim().isEmpty()) { + logger.error("parameter name cannot be null or blank"); + throw new IllegalArgumentException("parameter name cannot be null or blank"); + } + + PutParameterRequest request = new PutParameterRequest().withName(key).withValue(value).withOverwrite(true); + + PutParameterResult response = ssmClient.putParameter(request); + logger.info("Successfully updated parameter: " + key + " with value: " + value + ", Update Version: " + response.getVersion()); + return response.getVersion(); + } catch (Exception e) { + logger.error("Failed to update parameter: " + key + " with value: " + value, e); + throw new RuntimeException("Unexpected error updating parameter: " + key + " with value: " + value, e); + } + } + +} diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java index 0cb2483f24..7ad3205cb4 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; import com.bazaarvoice.emodb.sor.api.ChangeBuilder; @@ -18,6 +16,7 @@ import com.bazaarvoice.emodb.sor.db.test.InMemoryDataReaderDAO; import com.bazaarvoice.emodb.sor.delta.Delta; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.test.SystemClock; import com.bazaarvoice.emodb.sor.uuid.TimeUUIDs; import com.bazaarvoice.emodb.table.db.Table; @@ -487,7 +486,7 @@ public void compact(Table table, String key, UUID compactionKey, Compaction comp } }; - final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry(), new KafkaProducerService()); // Create a table for our test dataStore.createTable(tableName, @@ -573,7 +572,7 @@ public Record read(Key key, ReadConsistency ignored) { // Configure the data DAO to read 10 columns initially, causing other column reads to be read lazily dataDAO.setColumnBatchSize(10); - final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + final DataStore dataStore = new InMemoryDataStore(dataDAO, new MetricRegistry(), new KafkaProducerService()); // Create a table for our test dataStore.createTable(tableName, diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java index 6fabfcf1e4..5cff68b697 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; @@ -16,6 +14,7 @@ import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.delta.Delta; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.test.SystemClock; import com.bazaarvoice.emodb.sor.uuid.TimeUUIDs; import com.codahale.metrics.MetricRegistry; @@ -50,7 +49,7 @@ public class DataStoreTest { @Test public void testDeltas() throws Exception { - DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(),mock(BaseEventStore.class)); + DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); assertFalse(store.getTableExists(TABLE)); @@ -170,7 +169,7 @@ public void testDeltas() throws Exception { @Test public void testRecordTimestamps() throws Exception { - DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(),mock(BaseEventStore.class)); + DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); assertFalse(store.getTableExists(TABLE)); @@ -265,7 +264,7 @@ record = store.get(TABLE, KEY1); @Test public void testRecordTimestampsWithEventTags() throws Exception { - DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(),mock(BaseEventStore.class)); + DataStore store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); assertFalse(store.getTableExists(TABLE)); diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java index b32e194d91..57cba50ea7 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.TableOptionsBuilder; @@ -12,6 +10,7 @@ import com.bazaarvoice.emodb.sor.db.astyanax.ChangeEncoder; import com.bazaarvoice.emodb.sor.db.test.InMemoryDataReaderDAO; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.uuid.TimeUUIDs; import com.bazaarvoice.emodb.table.db.Table; import com.bazaarvoice.emodb.table.db.astyanax.PlacementCache; @@ -45,7 +44,7 @@ public List getSplits(Table table, int recordsPerSplit, int localResplit } }; - DataStore dataStore = new InMemoryDataStore(dataDao, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + DataStore dataStore = new InMemoryDataStore(dataDao, new MetricRegistry(), new KafkaProducerService()); dataStore.createTable("table", new TableOptionsBuilder().setPlacement("default").build(), Collections.emptyMap(), new AuditBuilder().build()); diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java index 8c14f5e7ee..06e2ce0993 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java @@ -2,8 +2,6 @@ import com.bazaarvoice.emodb.common.json.JsonHelper; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; @@ -26,6 +24,7 @@ import com.bazaarvoice.emodb.sor.db.test.DeltaClusteringKey; import com.bazaarvoice.emodb.sor.db.test.InMemoryDataReaderDAO; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.log.NullSlowQueryLog; import com.bazaarvoice.emodb.table.db.Table; import com.bazaarvoice.emodb.table.db.test.InMemoryTableDAO; @@ -67,7 +66,7 @@ public void testRedundantDeltas() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -124,7 +123,7 @@ public void testMinUUIDDelta() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -161,7 +160,7 @@ public void testRedundancyWithTags() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -242,7 +241,7 @@ public void testTagsForNestedMapDeltas() { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -262,7 +261,7 @@ public void testRedundancyWithCompactionAndUnchangedTag() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), new InMemoryTableDAO(), dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -339,7 +338,7 @@ public void testPartialCompactionWithNoRedundancy() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), tableDao, dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); @@ -411,7 +410,7 @@ public void testPartialCompactionWithRedundancy() throws Exception { DefaultDataStore store = new DefaultDataStore(new DatabusEventWriterRegistry(), tableDao, dataDao, dataDao, new NullSlowQueryLog(), new DiscardingExecutorService(), new InMemoryHistoryStore(), Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), new MetricRegistry(), Clock.systemUTC(), new KafkaProducerService()); TableOptions options = new TableOptionsBuilder().setPlacement("default").build(); store.createTable(TABLE, options, Collections.emptyMap(), newAudit("create table")); diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java index 765ce00d1d..3c1a7ce86e 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java @@ -1,8 +1,6 @@ package com.bazaarvoice.emodb.sor.core; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.TableOptionsBuilder; @@ -10,6 +8,7 @@ import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.db.test.InMemoryDataReaderDAO; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.test.SystemClock; import com.codahale.metrics.MetricRegistry; import com.google.common.collect.ImmutableMap; @@ -37,7 +36,7 @@ public class SorUpdateTest { public void SetupTest() { final InMemoryDataReaderDAO dataDAO = new InMemoryDataReaderDAO(); _eventWriterRegistry = new DatabusEventWriterRegistry(); - _dataStore = new InMemoryDataStore(_eventWriterRegistry, dataDAO, new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _dataStore = new InMemoryDataStore(_eventWriterRegistry, dataDAO, new MetricRegistry(), new KafkaProducerService()); // Create a table for our test diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java index 35f9d5c86c..d747a02798 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java @@ -1,8 +1,6 @@ package com.bazaarvoice.emodb.sor.test; import com.bazaarvoice.emodb.common.dropwizard.lifecycle.SimpleLifeCycleRegistry; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.audit.DiscardingAuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.InMemoryCompactionControlSource; @@ -13,6 +11,7 @@ import com.bazaarvoice.emodb.sor.core.test.InMemoryHistoryStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryMapStore; import com.bazaarvoice.emodb.sor.db.test.InMemoryDataReaderDAO; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.log.NullSlowQueryLog; import com.bazaarvoice.emodb.table.db.TableDAO; import com.bazaarvoice.emodb.table.db.test.InMemoryTableDAO; @@ -67,12 +66,12 @@ public MultiDCDataStores(int numDCs, boolean asyncCompacter, MetricRegistry metr if (asyncCompacter) { _stores[i] = new DefaultDataStore(new SimpleLifeCycleRegistry(), metricRegistry, new DatabusEventWriterRegistry(), _tableDao, _inMemoryDaos[i].setHistoryStore(_historyStores[i]), _replDaos[i], new NullSlowQueryLog(), _historyStores[i], - Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), new DiscardingAuditWriter(), new InMemoryMapStore<>(), Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), new DiscardingAuditWriter(), new InMemoryMapStore<>(), Clock.systemUTC(), new KafkaProducerService()); } else { _stores[i] = new DefaultDataStore(new DatabusEventWriterRegistry(), _tableDao, _inMemoryDaos[i].setHistoryStore(_historyStores[i]), _replDaos[i], new NullSlowQueryLog(), MoreExecutors.newDirectExecutorService(), _historyStores[i], Optional.empty(), new InMemoryCompactionControlSource(), Conditions.alwaysFalse(), - new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC(), new KafkaProducerService(), mock(BaseEventStore.class)); + new DiscardingAuditWriter(), new InMemoryMapStore<>(), metricRegistry, Clock.systemUTC(), new KafkaProducerService()); } } } diff --git a/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java b/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java index bb6009e0f2..2b5c157922 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java @@ -11,8 +11,6 @@ import com.bazaarvoice.emodb.common.zookeeper.store.ValueStore; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.core.DefaultDataCenter; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.FacadeExistsException; @@ -29,6 +27,7 @@ import com.bazaarvoice.emodb.sor.delta.Delta; import com.bazaarvoice.emodb.sor.delta.Deltas; import com.bazaarvoice.emodb.sor.delta.MapDeltaBuilder; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.table.db.MoveType; import com.bazaarvoice.emodb.table.db.Table; import com.bazaarvoice.emodb.table.db.TableBackingStore; @@ -1983,7 +1982,7 @@ dataCenter, mock(RateLimiterCache.class), dataCopyDAO, dataPurgeDAO, } private InMemoryDataStore newBackingStore(MetricRegistry metricRegistry) { - InMemoryDataStore store = new InMemoryDataStore(metricRegistry, new KafkaProducerService(), mock(BaseEventStore.class)); + InMemoryDataStore store = new InMemoryDataStore(metricRegistry, new KafkaProducerService()); store.createTable("__system:table", newOptions(PL_GLOBAL), ImmutableMap.of(), newAudit()); store.createTable("__system:table_uuid", newOptions(PL_GLOBAL), ImmutableMap.of(), newAudit()); store.createTable("__system:table_unpublished_databus_events", newOptions(PL_GLOBAL), ImmutableMap.of(), newAudit()); diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java index 4bdc43f9d5..65f8ad237c 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java @@ -2,7 +2,6 @@ import com.bazaarvoice.emodb.common.dropwizard.lifecycle.LifeCycleRegistry; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; import com.bazaarvoice.emodb.job.api.JobIdentifier; import com.bazaarvoice.emodb.job.api.JobRequest; import com.bazaarvoice.emodb.job.api.JobStatus; @@ -12,7 +11,6 @@ import com.bazaarvoice.emodb.job.handler.DefaultJobHandlerRegistry; import com.bazaarvoice.emodb.job.service.DefaultJobService; import com.bazaarvoice.emodb.queue.api.QueueService; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.CompactionControlSource; @@ -26,6 +24,7 @@ import com.bazaarvoice.emodb.sor.core.PurgeResult; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.web.resources.sor.AuditParam; import com.bazaarvoice.emodb.web.resources.sor.DataStoreResource1; import com.bazaarvoice.emodb.web.throttling.UnlimitedDataStoreUpdateThrottler; @@ -86,7 +85,7 @@ public void setUp() throws Exception { lifeCycleRegistry, _queueService, "testqueue", _jobHandlerRegistry, _jobStatusDAO, _curator, 1, Duration.ZERO, 100, Duration.ofHours(1)); - _store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _store = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); _dataStoreResource = new DataStoreResource1(_store, new DefaultDataStoreAsync(_store, _service, _jobHandlerRegistry), mock(CompactionControlSource.class), new UnlimitedDataStoreUpdateThrottler()); diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java index d5a4eaa125..a0693d3aec 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java @@ -12,11 +12,9 @@ import com.bazaarvoice.emodb.common.json.JsonHelper; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.api.DataCenters; -import com.bazaarvoice.emodb.event.api.BaseEventStore; import com.bazaarvoice.emodb.plugin.stash.StashMetadata; import com.bazaarvoice.emodb.plugin.stash.StashStateListener; import com.bazaarvoice.emodb.queue.core.ByteBufferInputStream; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.CompactionControlSource; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.api.ReadConsistency; @@ -29,6 +27,7 @@ import com.bazaarvoice.emodb.sor.db.Record; import com.bazaarvoice.emodb.sor.db.ScanRange; import com.bazaarvoice.emodb.sor.db.ScanRangeSplits; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.table.db.Table; import com.bazaarvoice.emodb.table.db.astyanax.AstyanaxStorage; import com.bazaarvoice.emodb.web.scanner.control.DistributedScanRangeMonitor; @@ -424,7 +423,7 @@ public void testScanUploadFromExistingScan() throws Exception { MetricRegistry metricRegistry = new MetricRegistry(); KafkaProducerService kafkaProducerService = new KafkaProducerService(); // Use an in-memory data store but override the default splits operation to return 4 splits for the test placement - InMemoryDataStore dataStore = spy(new InMemoryDataStore(metricRegistry, kafkaProducerService, mock(BaseEventStore.class))); + InMemoryDataStore dataStore = spy(new InMemoryDataStore(metricRegistry, kafkaProducerService)); when(dataStore.getScanRangeSplits("app_global:default", 1000000, Optional.empty())) .thenReturn(new ScanRangeSplits(ImmutableList.of( createSimpleSplitGroup("00", "40"), @@ -624,7 +623,7 @@ public void testScanFailureRecovery() Lists.newArrayList(), Lists.newArrayList()); InMemoryScanWorkflow scanWorkflow = new InMemoryScanWorkflow(); - ScanStatusDAO scanStatusDAO = new DataStoreScanStatusDAO(new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)), "scan_table", "app_global:sys"); + ScanStatusDAO scanStatusDAO = new DataStoreScanStatusDAO(new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()), "scan_table", "app_global:sys"); LocalScanUploadMonitor monitor = new LocalScanUploadMonitor(scanWorkflow, scanStatusDAO, mock(ScanWriterGenerator.class), mock(StashStateListener.class), mock(ScanCountListener.class), mock(DataTools.class), new InMemoryCompactionControlSource(), mock(DataCenters.class)); diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java index 2581db11fd..0ee31fe2ee 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java @@ -1,13 +1,12 @@ package com.bazaarvoice.emodb.web.scanner.scanstatus; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.db.ScanRange; import com.bazaarvoice.emodb.sor.delta.Deltas; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.web.scanner.ScanDestination; import com.bazaarvoice.emodb.web.scanner.ScanOptions; import com.codahale.metrics.MetricRegistry; @@ -36,7 +35,7 @@ public class DataStoreScanStatusDAOTest { @BeforeMethod public void setUp() { - _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); _dao = new DataStoreScanStatusDAO(_dataStore, "scan_table", "app_global:sys"); } diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java index a5df6effb3..04414530a9 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java @@ -1,9 +1,8 @@ package com.bazaarvoice.emodb.web.scanner.scanstatus; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.codahale.metrics.MetricRegistry; import com.google.common.collect.ImmutableSet; import org.testng.annotations.BeforeMethod; @@ -22,7 +21,7 @@ public class DataStoreStashRequestDAOTest { @BeforeMethod public void setUp() { - _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); _dao = new DataStoreStashRequestDAO(_dataStore, "request_table", "app_global:sys"); } diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java index 1ad088d5ee..a213da5ae4 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java @@ -3,11 +3,10 @@ import com.bazaarvoice.emodb.cachemgr.api.CacheHandle; import com.bazaarvoice.emodb.cachemgr.api.CacheRegistry; import com.bazaarvoice.emodb.cachemgr.api.InvalidationScope; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.codahale.metrics.MetricRegistry; import com.google.common.cache.Cache; import com.google.inject.util.Providers; @@ -34,7 +33,7 @@ public class SettingsManagerTest { @BeforeMethod public void setUp() { - _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService(), mock(BaseEventStore.class)); + _dataStore = new InMemoryDataStore(new MetricRegistry(), new KafkaProducerService()); _cacheRegistry = mock(CacheRegistry.class); _cacheHandle = mock(CacheHandle.class); when(_cacheRegistry.register(eq("settings"), any(Cache.class), eq(true))).thenReturn(_cacheHandle); From bd75df233de4ae293f1de68cf5041b96b2d21663 Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Tue, 5 Nov 2024 09:50:48 +0000 Subject: [PATCH 3/7] removed redundant dependencies --- .../test/integration/auth/TableAuthIdentityManagerDAOTest.java | 2 -- .../java/test/integration/auth/TableRoleManagerDAOTest.java | 2 -- .../src/test/java/test/integration/sor/CasStashTableTest.java | 2 -- .../com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java | 2 -- .../test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java | 2 -- .../test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java | 2 -- .../java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java | 2 -- .../java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java | 2 -- .../test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java | 2 -- .../java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java | 2 -- .../bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java | 2 -- .../test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java | 2 -- .../com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java | 2 -- .../web/scanner/scanstatus/DataStoreScanStatusDAOTest.java | 2 -- .../web/scanner/scanstatus/DataStoreStashRequestDAOTest.java | 2 -- .../com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java | 2 -- 16 files changed, 32 deletions(-) diff --git a/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java b/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java index faec7f1759..e6da43c029 100644 --- a/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java +++ b/quality/integration/src/test/java/test/integration/auth/TableAuthIdentityManagerDAOTest.java @@ -3,8 +3,6 @@ import com.bazaarvoice.emodb.auth.apikey.ApiKey; import com.bazaarvoice.emodb.auth.apikey.ApiKeyModification; import com.bazaarvoice.emodb.auth.identity.TableAuthIdentityManagerDAO; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; diff --git a/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java b/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java index 5f5b77a2f3..a3db269d93 100644 --- a/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java +++ b/quality/integration/src/test/java/test/integration/auth/TableRoleManagerDAOTest.java @@ -9,8 +9,6 @@ import com.bazaarvoice.emodb.auth.role.RoleModification; import com.bazaarvoice.emodb.auth.role.RoleNotFoundException; import com.bazaarvoice.emodb.auth.role.TableRoleManagerDAO; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; diff --git a/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java b/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java index abca4f7a8f..79ce74873f 100644 --- a/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java +++ b/quality/integration/src/test/java/test/integration/sor/CasStashTableTest.java @@ -10,8 +10,6 @@ import com.bazaarvoice.emodb.common.zookeeper.store.ValueStore; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.api.DataCenters; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.condition.Conditions; diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java index 6f98145ef6..8f6d9da161 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/test/InMemoryDataStore.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core.test; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.audit.DiscardingAuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.InMemoryCompactionControlSource; import com.bazaarvoice.emodb.sor.condition.Conditions; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java index 6ace82276a..7ad3205cb4 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/CompactorTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; import com.bazaarvoice.emodb.sor.api.ChangeBuilder; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java index 0355186a57..5cff68b697 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/DataStoreTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java index 2039d689fe..57cba50ea7 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/MinSplitSizeTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.sor.core; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.TableOptionsBuilder; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java index 61b705415b..06e2ce0993 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/RedundantDeltaTest.java @@ -2,8 +2,6 @@ import com.bazaarvoice.emodb.common.json.JsonHelper; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.Change; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java index 6f5305b7cd..3c1a7ce86e 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/core/SorUpdateTest.java @@ -1,8 +1,6 @@ package com.bazaarvoice.emodb.sor.core; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.TableOptionsBuilder; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java b/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java index 52face82ba..d747a02798 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/sor/test/MultiDCDataStores.java @@ -1,8 +1,6 @@ package com.bazaarvoice.emodb.sor.test; import com.bazaarvoice.emodb.common.dropwizard.lifecycle.SimpleLifeCycleRegistry; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.audit.DiscardingAuditWriter; import com.bazaarvoice.emodb.sor.compactioncontrol.InMemoryCompactionControlSource; diff --git a/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java b/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java index d0a3b77b40..2b5c157922 100644 --- a/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java +++ b/sor/src/test/java/com/bazaarvoice/emodb/table/db/astyanax/TableLifeCycleTest.java @@ -11,8 +11,6 @@ import com.bazaarvoice.emodb.common.zookeeper.store.ValueStore; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.core.DefaultDataCenter; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.FacadeExistsException; diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java index d43c673c09..65f8ad237c 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/purge/PurgeTest.java @@ -2,7 +2,6 @@ import com.bazaarvoice.emodb.common.dropwizard.lifecycle.LifeCycleRegistry; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; import com.bazaarvoice.emodb.job.api.JobIdentifier; import com.bazaarvoice.emodb.job.api.JobRequest; import com.bazaarvoice.emodb.job.api.JobStatus; @@ -12,7 +11,6 @@ import com.bazaarvoice.emodb.job.handler.DefaultJobHandlerRegistry; import com.bazaarvoice.emodb.job.service.DefaultJobService; import com.bazaarvoice.emodb.queue.api.QueueService; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.Audit; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.CompactionControlSource; diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java index 9e0d67b8b5..a0693d3aec 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/ScanUploaderTest.java @@ -12,11 +12,9 @@ import com.bazaarvoice.emodb.common.json.JsonHelper; import com.bazaarvoice.emodb.datacenter.api.DataCenter; import com.bazaarvoice.emodb.datacenter.api.DataCenters; -import com.bazaarvoice.emodb.event.api.BaseEventStore; import com.bazaarvoice.emodb.plugin.stash.StashMetadata; import com.bazaarvoice.emodb.plugin.stash.StashStateListener; import com.bazaarvoice.emodb.queue.core.ByteBufferInputStream; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.CompactionControlSource; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.api.ReadConsistency; diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java index fd91cbb160..0ee31fe2ee 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreScanStatusDAOTest.java @@ -1,8 +1,6 @@ package com.bazaarvoice.emodb.web.scanner.scanstatus; import com.bazaarvoice.emodb.common.uuid.TimeUUIDs; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.AuditBuilder; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java index 83bd716e53..04414530a9 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/scanner/scanstatus/DataStoreStashRequestDAOTest.java @@ -1,7 +1,5 @@ package com.bazaarvoice.emodb.web.scanner.scanstatus; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; diff --git a/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java b/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java index ed9d240c4c..a213da5ae4 100644 --- a/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java +++ b/web/src/test/java/com/bazaarvoice/emodb/web/settings/SettingsManagerTest.java @@ -3,8 +3,6 @@ import com.bazaarvoice.emodb.cachemgr.api.CacheHandle; import com.bazaarvoice.emodb.cachemgr.api.CacheRegistry; import com.bazaarvoice.emodb.cachemgr.api.InvalidationScope; -import com.bazaarvoice.emodb.event.api.BaseEventStore; -import com.bazaarvoice.emodb.queue.core.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.api.DataStore; import com.bazaarvoice.emodb.sor.api.Intrinsic; import com.bazaarvoice.emodb.sor.core.test.InMemoryDataStore; From 17307f3c6febfac49803f66748f4f929a3103efc Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Tue, 5 Nov 2024 13:56:34 +0000 Subject: [PATCH 4/7] additional changes --- .../bazaarvoice/emodb/queue/QueueModule.java | 2 + .../emodb/sor/DataStoreModule.java | 2 + .../emodb/sor/core/DefaultDataStore.java | 4 +- .../emodb/sor/kafka/KafkaAdminService.java | 118 ------------------ .../emodb/sor/kafka/KafkaConfig.java | 9 -- .../emodb/sor/kafka/KafkaProducerService.java | 32 +++++ 6 files changed, 39 insertions(+), 128 deletions(-) delete mode 100644 sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java diff --git a/queue/src/main/java/com/bazaarvoice/emodb/queue/QueueModule.java b/queue/src/main/java/com/bazaarvoice/emodb/queue/QueueModule.java index b42d33b02c..6cbc958fe0 100644 --- a/queue/src/main/java/com/bazaarvoice/emodb/queue/QueueModule.java +++ b/queue/src/main/java/com/bazaarvoice/emodb/queue/QueueModule.java @@ -34,6 +34,7 @@ import com.google.inject.Provides; import com.google.inject.Singleton; import com.google.inject.TypeLiteral; +import com.timgroup.statsd.StatsDClient; import org.apache.curator.framework.CuratorFramework; import java.time.Clock; @@ -76,6 +77,7 @@ public QueueModule(MetricRegistry metricRegistry) { @Override protected void configure() { bind(CassandraFactory.class).asEagerSingleton(); + bind(StatsDClient.class).asEagerSingleton(); // Event Store bind(ChannelConfiguration.class).to(QueueChannelConfiguration.class).asEagerSingleton(); diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/DataStoreModule.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/DataStoreModule.java index fda127a190..5c5faffebc 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/DataStoreModule.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/DataStoreModule.java @@ -60,6 +60,7 @@ import com.bazaarvoice.emodb.sor.db.cql.CqlForMultiGets; import com.bazaarvoice.emodb.sor.db.cql.CqlForScans; import com.bazaarvoice.emodb.sor.db.cql.SorCqlSettingsTask; +import com.bazaarvoice.emodb.sor.kafka.KafkaProducerService; import com.bazaarvoice.emodb.sor.log.LogbackSlowQueryLogProvider; import com.bazaarvoice.emodb.sor.log.SlowQueryLog; import com.bazaarvoice.emodb.sor.log.SlowQueryLogConfiguration; @@ -195,6 +196,7 @@ protected void configure() { bind(SlowQueryLog.class).toProvider(LogbackSlowQueryLogProvider.class); bind(HintsConsistencyTimeProvider.class).asEagerSingleton(); bind(MinLagConsistencyTimeProvider.class).asEagerSingleton(); + bind(KafkaProducerService.class); // The web servers are responsible for updating the ZooKeeper full consistency data. CLI tools don't need to. // Enable updating the ZooKeeper full consistency data if specified diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java index 4b6475b598..ac20c5d70a 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java @@ -56,6 +56,8 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; +import java.io.BufferedReader; +import java.io.FileReader; import java.io.IOException; import java.net.URI; import java.time.Clock; @@ -81,7 +83,7 @@ public class DefaultDataStore implements DataStore, DataProvider, DataTools, Tab private static final int MAX_COMPACTION_QUEUE_LENGTH = 100; private static final String SYSTEM_PREFIX = "__system_bus:"; private static final String MASTER_FANOUT = SYSTEM_PREFIX + "master"; - private static final String UNIVERSE = KafkaConfig.getUniverseFromEnv(); + private static final String UNIVERSE = KafkaProducerService.getUniverseFromEnv(); private static final String DATA_THROTTLER = "databusThrottler"; private final Logger _log = LoggerFactory.getLogger(DefaultDataStore.class); diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java deleted file mode 100644 index 8bdccc2918..0000000000 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java +++ /dev/null @@ -1,118 +0,0 @@ -package com.bazaarvoice.emodb.sor.kafka; - -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.NewTopic; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; - -public class KafkaAdminService { - private static final Logger _log = LoggerFactory.getLogger(KafkaAdminService.class); - private final AdminClient adminClient; - // Cache for the list of all topics with a TTL of 10 minutes - private final Cache> topicListCache = CacheBuilder.newBuilder() - .expireAfterWrite(1, TimeUnit.MINUTES) - .build(); - - private static final String TOPIC_LIST_KEY = "allTopics"; - - - public KafkaAdminService() { - this.adminClient = AdminClient.create(KafkaConfig.getAdminProps()); - } - - /** - * Creates a new Kafka topic with the specified configurations. - * - * @param topic The name of the topic. - * @param numPartitions Number of partitions. - * @param replicationFactor Replication factor. - */ - public Boolean createTopicIfNotExists(String topic, int numPartitions, short replicationFactor, String queueType) { - Boolean isExisting =isTopicExists(topic); - if (! isExisting) { - //create the topic now - NewTopic newTopic = new NewTopic(topic, numPartitions, replicationFactor); - try { - adminClient.createTopics(Collections.singleton(newTopic)).all().get(); - addToCache(topic); - _log.info("Created topic: {} with numPartitions: {} and replication factor {} ", topic, numPartitions, replicationFactor); - } catch (Exception e) { - _log.error("Error creating topic {}: ", topic, e); - throw new RuntimeException(e); - } - } - return isExisting; - } - public void addToCache(String topic){ - Set topics = topicListCache.getIfPresent(TOPIC_LIST_KEY); - if (topics == null) { - topics = new HashSet<>(); - } else { - // Create a new mutable Set if the existing one is unmodifiable - topics = new HashSet<>(topics); - } - topics.add(topic); - topicListCache.put(TOPIC_LIST_KEY, topics); - _log.info("Added newly created topic to cache: {}", topic); - } - - - /** - * Checks if a Kafka topic exists by using a cache to store the list of all topics. - * If the cache entry has expired or the cache is empty, it queries the Kafka AdminClient for the topic list. - *

- * The cached list has a TTL (Time-To-Live) of 10 minutes, after which it will be refreshed - * from Kafka on the next access. - *

- * - * @param topic the name of the Kafka topic to check - * @return {@code true} if the topic exists, otherwise {@code false}. - * @throws RuntimeException if there is an error fetching the topic list or checking if the topic exists. - */ - public boolean isTopicExists(String topic) { - try { - // Retrieve the list of topics from the cache - Set topics = topicListCache.get(TOPIC_LIST_KEY, this::fetchTopicListFromKafka); - - // Check if the given topic is in the cached list - return topics.contains(topic); - } catch (ExecutionException e) { - _log.error("Failed to check if topic exists: {}", topic, e); - throw new RuntimeException("Error checking if topic exists", e); - } - } - - /** - * Fetches the list of all topic names from Kafka AdminClient. - * This method is called only when the cache is expired or empty. - * - * @return a Set containing all topic names. - * @throws ExecutionException if there is an error fetching the topic list from Kafka. - */ - private Set fetchTopicListFromKafka() throws ExecutionException { - try { - _log.info("Fetching topic list from Kafka"); - return adminClient.listTopics().names().get(); - } catch (Exception e) { - _log.error("Error fetching topic list from Kafka", e); - throw new ExecutionException(e); - } - } - - /** - * Closes the AdminClient to release resources. - */ - public void close() { - if (adminClient != null) { - adminClient.close(); - } - } -} \ No newline at end of file diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java index 255bc53fb8..be76309a2b 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaConfig.java @@ -134,15 +134,6 @@ public static Properties getProducerProps() { return producerProps; } - // Kafka Admin properties - public static Properties getAdminProps() { - Properties adminProps = new Properties(); - - adminProps.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersConfig); - logger.info("Kafka Admin properties initialized."); - return adminProps; - } - // Ensure the SSM client is closed when the application shuts down public static void shutdown() { if (ssmClient != null) { diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java index 4cb587cf29..c8be5c8046 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java @@ -6,11 +6,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.BufferedReader; +import java.io.FileReader; +import java.io.IOException; import java.time.Duration; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Properties; import java.util.concurrent.Future; public class KafkaProducerService { @@ -63,4 +67,32 @@ public void close() { throw e; } } + + public static String getUniverseFromEnv() { + String filePath = "/etc/environment"; + Properties environmentProps = new Properties(); + + try (BufferedReader reader = new BufferedReader(new FileReader(filePath))) { + String line; + while ((line = reader.readLine()) != null) { + // Skip empty lines or comments + if (line.trim().isEmpty() || line.trim().startsWith("#")) { + continue; + } + // Split the line into key-value pair + String[] parts = line.split("=", 2); + if (parts.length == 2) { + String key = parts[0].trim(); + String value = parts[1].trim(); + // Remove any surrounding quotes from value + value = value.replace("\"", ""); + environmentProps.put(key, value); + } + } + // Access the environment variables + return environmentProps.getProperty("UNIVERSE"); + } catch (IOException e) { + throw new RuntimeException("Error reading environment file: " + e.getMessage()); + } + } } \ No newline at end of file From 3d36797fa9f8906ece21c0a5c9566415373a5476 Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Tue, 5 Nov 2024 13:58:26 +0000 Subject: [PATCH 5/7] additional changes --- .../emodb/sor/kafka/KafkaAdminService.java | 118 ------------------ 1 file changed, 118 deletions(-) delete mode 100644 sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java deleted file mode 100644 index 8bdccc2918..0000000000 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaAdminService.java +++ /dev/null @@ -1,118 +0,0 @@ -package com.bazaarvoice.emodb.sor.kafka; - -import com.google.common.cache.Cache; -import com.google.common.cache.CacheBuilder; -import org.apache.kafka.clients.admin.AdminClient; -import org.apache.kafka.clients.admin.NewTopic; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; - -public class KafkaAdminService { - private static final Logger _log = LoggerFactory.getLogger(KafkaAdminService.class); - private final AdminClient adminClient; - // Cache for the list of all topics with a TTL of 10 minutes - private final Cache> topicListCache = CacheBuilder.newBuilder() - .expireAfterWrite(1, TimeUnit.MINUTES) - .build(); - - private static final String TOPIC_LIST_KEY = "allTopics"; - - - public KafkaAdminService() { - this.adminClient = AdminClient.create(KafkaConfig.getAdminProps()); - } - - /** - * Creates a new Kafka topic with the specified configurations. - * - * @param topic The name of the topic. - * @param numPartitions Number of partitions. - * @param replicationFactor Replication factor. - */ - public Boolean createTopicIfNotExists(String topic, int numPartitions, short replicationFactor, String queueType) { - Boolean isExisting =isTopicExists(topic); - if (! isExisting) { - //create the topic now - NewTopic newTopic = new NewTopic(topic, numPartitions, replicationFactor); - try { - adminClient.createTopics(Collections.singleton(newTopic)).all().get(); - addToCache(topic); - _log.info("Created topic: {} with numPartitions: {} and replication factor {} ", topic, numPartitions, replicationFactor); - } catch (Exception e) { - _log.error("Error creating topic {}: ", topic, e); - throw new RuntimeException(e); - } - } - return isExisting; - } - public void addToCache(String topic){ - Set topics = topicListCache.getIfPresent(TOPIC_LIST_KEY); - if (topics == null) { - topics = new HashSet<>(); - } else { - // Create a new mutable Set if the existing one is unmodifiable - topics = new HashSet<>(topics); - } - topics.add(topic); - topicListCache.put(TOPIC_LIST_KEY, topics); - _log.info("Added newly created topic to cache: {}", topic); - } - - - /** - * Checks if a Kafka topic exists by using a cache to store the list of all topics. - * If the cache entry has expired or the cache is empty, it queries the Kafka AdminClient for the topic list. - *

- * The cached list has a TTL (Time-To-Live) of 10 minutes, after which it will be refreshed - * from Kafka on the next access. - *

- * - * @param topic the name of the Kafka topic to check - * @return {@code true} if the topic exists, otherwise {@code false}. - * @throws RuntimeException if there is an error fetching the topic list or checking if the topic exists. - */ - public boolean isTopicExists(String topic) { - try { - // Retrieve the list of topics from the cache - Set topics = topicListCache.get(TOPIC_LIST_KEY, this::fetchTopicListFromKafka); - - // Check if the given topic is in the cached list - return topics.contains(topic); - } catch (ExecutionException e) { - _log.error("Failed to check if topic exists: {}", topic, e); - throw new RuntimeException("Error checking if topic exists", e); - } - } - - /** - * Fetches the list of all topic names from Kafka AdminClient. - * This method is called only when the cache is expired or empty. - * - * @return a Set containing all topic names. - * @throws ExecutionException if there is an error fetching the topic list from Kafka. - */ - private Set fetchTopicListFromKafka() throws ExecutionException { - try { - _log.info("Fetching topic list from Kafka"); - return adminClient.listTopics().names().get(); - } catch (Exception e) { - _log.error("Error fetching topic list from Kafka", e); - throw new ExecutionException(e); - } - } - - /** - * Closes the AdminClient to release resources. - */ - public void close() { - if (adminClient != null) { - adminClient.close(); - } - } -} \ No newline at end of file From a6a7bf3776a3078d4751ca991f737ccfc485016f Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Tue, 5 Nov 2024 14:03:19 +0000 Subject: [PATCH 6/7] additional changes --- .../emodb/sor/core/DefaultDataStore.java | 2 +- .../emodb/sor/kafka/KafkaProducerService.java | 28 ------------------- 2 files changed, 1 insertion(+), 29 deletions(-) diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java index ac20c5d70a..bf8a3fcf48 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java @@ -83,7 +83,7 @@ public class DefaultDataStore implements DataStore, DataProvider, DataTools, Tab private static final int MAX_COMPACTION_QUEUE_LENGTH = 100; private static final String SYSTEM_PREFIX = "__system_bus:"; private static final String MASTER_FANOUT = SYSTEM_PREFIX + "master"; - private static final String UNIVERSE = KafkaProducerService.getUniverseFromEnv(); + private static final String UNIVERSE = KafkaConfig.getUniverseFromEnv(); private static final String DATA_THROTTLER = "databusThrottler"; private final Logger _log = LoggerFactory.getLogger(DefaultDataStore.class); diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java index c8be5c8046..4905b4dc9e 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java @@ -67,32 +67,4 @@ public void close() { throw e; } } - - public static String getUniverseFromEnv() { - String filePath = "/etc/environment"; - Properties environmentProps = new Properties(); - - try (BufferedReader reader = new BufferedReader(new FileReader(filePath))) { - String line; - while ((line = reader.readLine()) != null) { - // Skip empty lines or comments - if (line.trim().isEmpty() || line.trim().startsWith("#")) { - continue; - } - // Split the line into key-value pair - String[] parts = line.split("=", 2); - if (parts.length == 2) { - String key = parts[0].trim(); - String value = parts[1].trim(); - // Remove any surrounding quotes from value - value = value.replace("\"", ""); - environmentProps.put(key, value); - } - } - // Access the environment variables - return environmentProps.getProperty("UNIVERSE"); - } catch (IOException e) { - throw new RuntimeException("Error reading environment file: " + e.getMessage()); - } - } } \ No newline at end of file From decbf2e7f1cc387205b7e277733b02f175d3fd38 Mon Sep 17 00:00:00 2001 From: anurag-dubey_bveng Date: Tue, 5 Nov 2024 14:06:38 +0000 Subject: [PATCH 7/7] removed unneccessary dependencies --- .../java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java | 2 -- .../com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java | 4 ---- 2 files changed, 6 deletions(-) diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java index bf8a3fcf48..4b6475b598 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/core/DefaultDataStore.java @@ -56,8 +56,6 @@ import javax.annotation.Nullable; import javax.validation.constraints.NotNull; -import java.io.BufferedReader; -import java.io.FileReader; import java.io.IOException; import java.net.URI; import java.time.Clock; diff --git a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java index 4905b4dc9e..4cb587cf29 100644 --- a/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java +++ b/sor/src/main/java/com/bazaarvoice/emodb/sor/kafka/KafkaProducerService.java @@ -6,15 +6,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedReader; -import java.io.FileReader; -import java.io.IOException; import java.time.Duration; import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Properties; import java.util.concurrent.Future; public class KafkaProducerService {