From 39740b7ba918f436e598b006697befa96ebaf631 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 15 Feb 2019 15:37:50 -0800 Subject: [PATCH 01/15] Rename InMemoryDB to RocksDB --- ...kupRocksDBExtractionNamespaceFactory.java} | 10 +-- ...actor.java => RocksDBLookupExtractor.java} | 12 +-- .../lookup/namespace/ExtractionNamespace.java | 2 +- ...e.java => RocksDBExtractionNamespace.java} | 10 +-- .../server/lookup/namespace/KafkaManager.java | 7 +- .../MahaNamespaceExtractionModule.java | 6 +- ...cksDBExtractionNamespaceCacheFactory.java} | 18 ++--- .../lookup/namespace/RocksDBManager.java | 13 ++-- .../cache/MahaExtractionCacheManager.java | 9 +-- .../lookup/MissingLookupManagerTest.java | 11 ++- ...t.java => RocksDBLookupExtractorTest.java} | 74 +++++++++---------- ...BExtractionNamespaceCacheFactoryTest.java} | 16 ++-- 12 files changed, 91 insertions(+), 97 deletions(-) rename druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/{MissingLookupInMemoryDBExtractionNamespaceFactory.java => MissingLookupRocksDBExtractionNamespaceFactory.java} (92%) rename druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/{InMemoryDBLookupExtractor.java => RocksDBLookupExtractor.java} (95%) rename druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/{InMemoryDBExtractionNamespace.java => RocksDBExtractionNamespace.java} (93%) rename druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/{InMemoryDBExtractionNamespaceCacheFactory.java => RocksDBExtractionNamespaceCacheFactory.java} (88%) rename druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/{InMemoryDBLookupExtractorTest.java => RocksDBLookupExtractorTest.java} (76%) rename druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/{InMemoryDBExtractionNamespaceCacheFactoryTest.java => RocksDBExtractionNamespaceCacheFactoryTest.java} (93%) diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupInMemoryDBExtractionNamespaceFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupRocksDBExtractionNamespaceFactory.java similarity index 92% rename from druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupInMemoryDBExtractionNamespaceFactory.java rename to druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupRocksDBExtractionNamespaceFactory.java index 800020b59..4e89066a5 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupInMemoryDBExtractionNamespaceFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupRocksDBExtractionNamespaceFactory.java @@ -7,7 +7,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.Message; import com.metamx.common.logger.Logger; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.PasswordProvider; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; import org.apache.kafka.clients.producer.KafkaProducer; @@ -19,10 +19,10 @@ import java.util.Map; import java.util.Properties; -public class MissingLookupInMemoryDBExtractionNamespaceFactory implements +public class MissingLookupRocksDBExtractionNamespaceFactory implements MissingLookupExtractionNamespaceFactory { - private static final Logger LOGGER = new Logger(MissingLookupInMemoryDBExtractionNamespaceFactory.class); + private static final Logger LOGGER = new Logger(MissingLookupRocksDBExtractionNamespaceFactory.class); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); private DBI dbi = null; private KafkaProducer kafkaProducer = null; @@ -34,7 +34,7 @@ public void process(String dimension, Properties kafkaProperties, String producerKafkaTopic) throws IOException { - InMemoryDBExtractionNamespace extractionNamespace = OBJECT_MAPPER.readValue(extractionNamespaceByteArray, InMemoryDBExtractionNamespace.class); + RocksDBExtractionNamespace extractionNamespace = OBJECT_MAPPER.readValue(extractionNamespaceByteArray, RocksDBExtractionNamespace.class); dbi = ensureDBI(passwordProvider, extractionNamespace); kafkaProducer = ensureKafkaProducer(kafkaProperties); dbi.withHandle( handle -> { @@ -66,7 +66,7 @@ public void process(String dimension, }); } - synchronized DBI ensureDBI(PasswordProvider passwordProvider, InMemoryDBExtractionNamespace namespace) { + synchronized DBI ensureDBI(PasswordProvider passwordProvider, RocksDBExtractionNamespace namespace) { if (dbi == null) { dbi = new DBI( namespace.getMissingLookupConfig().getConnectorConfig().getConnectURI(), diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/InMemoryDBLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java similarity index 95% rename from druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/InMemoryDBLookupExtractor.java rename to druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java index 0c098d405..3b4887dd7 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/InMemoryDBLookupExtractor.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java @@ -19,7 +19,7 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.KafkaManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.MonitoringConstants; @@ -39,12 +39,12 @@ import static java.nio.charset.StandardCharsets.UTF_8; -public class InMemoryDBLookupExtractor extends LookupExtractor +public class RocksDBLookupExtractor extends LookupExtractor { - private static final Logger LOG = new Logger(InMemoryDBLookupExtractor.class); + private static final Logger LOG = new Logger(RocksDBLookupExtractor.class); private static final ObjectMapper objectMapper = new ObjectMapper(); private final Map map; - private final InMemoryDBExtractionNamespace extractionNamespace; + private final RocksDBExtractionNamespace extractionNamespace; private RocksDBManager rocksDBManager; private LookupService lookupService; private ProtobufSchemaFactory protobufSchemaFactory; @@ -53,7 +53,7 @@ public class InMemoryDBLookupExtractor extends LookupExtractor private Cache missingLookupCache; private final byte[] extractionNamespaceAsByteArray; - public InMemoryDBLookupExtractor(InMemoryDBExtractionNamespace extractionNamespace, Map map, + public RocksDBLookupExtractor(RocksDBExtractionNamespace extractionNamespace, Map map, LookupService lookupService, RocksDBManager rocksDBManager, KafkaManager kafkaManager, ProtobufSchemaFactory protobufSchemaFactory, ServiceEmitter serviceEmitter) { this.extractionNamespace = extractionNamespace; @@ -203,7 +203,7 @@ public boolean equals(Object o) return false; } - InMemoryDBLookupExtractor that = (InMemoryDBLookupExtractor) o; + RocksDBLookupExtractor that = (RocksDBLookupExtractor) o; return map.equals(that.map); } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java index fd529da92..3b13733a1 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java @@ -8,7 +8,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "mahajdbc", value = JDBCExtractionNamespace.class), - @JsonSubTypes.Type(name = "mahainmemorydb", value = InMemoryDBExtractionNamespace.class) + @JsonSubTypes.Type(name = "maharocksdb", value = RocksDBExtractionNamespace.class) }) public interface ExtractionNamespace { diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/InMemoryDBExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java similarity index 93% rename from druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/InMemoryDBExtractionNamespace.java rename to druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java index 94763677d..1ff7c4ef6 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/InMemoryDBExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java @@ -13,8 +13,8 @@ import javax.validation.constraints.NotNull; import java.util.Objects; -@JsonTypeName("mahainmemorydb") -public class InMemoryDBExtractionNamespace implements ExtractionNamespace { +@JsonTypeName("maharocksdb") +public class RocksDBExtractionNamespace implements ExtractionNamespace { @JsonProperty private final String rocksDbInstanceHDFSPath; @@ -40,7 +40,7 @@ public class InMemoryDBExtractionNamespace implements ExtractionNamespace { private Long lastUpdatedTime = -1L; @JsonCreator - public InMemoryDBExtractionNamespace(@NotNull @JsonProperty(value = "namespace", required = true) + public RocksDBExtractionNamespace(@NotNull @JsonProperty(value = "namespace", required = true) String namespace, @NotNull @JsonProperty(value = "rocksDbInstanceHDFSPath", required = true) final String rocksDbInstanceHDFSPath, @@ -128,7 +128,7 @@ public String getTsColumn() @Override public String toString() { return String.format( - "InMemoryDBExtractionNamespace = { namespace = %s, rocksDbInstanceHDFSPath = { %s }, pollPeriod = %s, kafkaTopic = %s, missingLookupConfig = %s }", + "RocksDBExtractionNamespace = { namespace = %s, rocksDbInstanceHDFSPath = { %s }, pollPeriod = %s, kafkaTopic = %s, missingLookupConfig = %s }", namespace, rocksDbInstanceHDFSPath, pollPeriod, @@ -141,7 +141,7 @@ public String toString() { public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - InMemoryDBExtractionNamespace that = (InMemoryDBExtractionNamespace) o; + RocksDBExtractionNamespace that = (RocksDBExtractionNamespace) o; return cacheEnabled == that.cacheEnabled && lookupAuditingEnabled == that.lookupAuditingEnabled && Objects.equals(rocksDbInstanceHDFSPath, that.rocksDbInstanceHDFSPath) && diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java index 88912e997..60bef732c 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java @@ -5,7 +5,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.google.inject.Inject; import com.google.inject.Provider; -import com.google.inject.name.Named; import com.google.protobuf.Descriptors; import com.google.protobuf.InvalidProtocolBufferException; import com.google.protobuf.Message; @@ -14,7 +13,7 @@ import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; import io.druid.guice.ManageLifecycle; @@ -92,7 +91,7 @@ private void updateRocksDB(final Parser parser, final Descriptors.Descr } } - public void applyChangesSinceBeginning(final InMemoryDBExtractionNamespace extractionNamespace, + public void applyChangesSinceBeginning(final RocksDBExtractionNamespace extractionNamespace, final String groupId, final RocksDB rocksDB, final ConcurrentMap kafkaPartitionOffset) { final String topic = extractionNamespace.getKafkaTopic(); @@ -173,7 +172,7 @@ public Boolean call() { log.info("Applied all the changes since the beginning [%s]", topic); } - public void addListener(final InMemoryDBExtractionNamespace kafkaNamespace, final String groupId, + public void addListener(final RocksDBExtractionNamespace kafkaNamespace, final String groupId, final ConcurrentMap kafkaPartitionOffset, final boolean seekOffsetToPreviousSnapshot) { final String topic = kafkaNamespace.getKafkaTopic(); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java index fc1a59770..fd0796951 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java @@ -13,7 +13,7 @@ import com.yahoo.maha.maha_druid_lookups.query.lookup.MahaRegisteredLookupExtractionFn; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaExtractionCacheManager; @@ -84,8 +84,8 @@ public void configure(Binder binder) .to(JDBCExtractionNamespaceCacheFactory.class) .in(LazySingleton.class); getNamespaceFactoryMapBinder(binder) - .addBinding(InMemoryDBExtractionNamespace.class) - .to(InMemoryDBExtractionNamespaceCacheFactory.class) + .addBinding(RocksDBExtractionNamespace.class) + .to(RocksDBExtractionNamespaceCacheFactory.class) .in(LazySingleton.class); LifecycleModule.register(binder, RocksDBManager.class); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/InMemoryDBExtractionNamespaceCacheFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java similarity index 88% rename from druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/InMemoryDBExtractionNamespaceCacheFactory.java rename to druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java index a3ba65a03..3a8d028c5 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/InMemoryDBExtractionNamespaceCacheFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java @@ -11,7 +11,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; @@ -23,10 +23,10 @@ /** * */ -public class InMemoryDBExtractionNamespaceCacheFactory - implements ExtractionNamespaceCacheFactory +public class RocksDBExtractionNamespaceCacheFactory + implements ExtractionNamespaceCacheFactory { - private static final Logger LOG = new Logger(InMemoryDBExtractionNamespaceCacheFactory.class); + private static final Logger LOG = new Logger(RocksDBExtractionNamespaceCacheFactory.class); private static final String ZERO = "0"; @Inject LookupService lookupService; @@ -40,7 +40,7 @@ public class InMemoryDBExtractionNamespaceCacheFactory @Override public Callable getCachePopulator( final String id, - final InMemoryDBExtractionNamespace extractionNamespace, + final RocksDBExtractionNamespace extractionNamespace, final String lastVersion, final Map cache ) @@ -70,7 +70,7 @@ public String call() { } @Override - public void updateCache(final InMemoryDBExtractionNamespace extractionNamespace, + public void updateCache(final RocksDBExtractionNamespace extractionNamespace, final Map cache, final String key, final byte[] value) { if (extractionNamespace.isCacheEnabled()) { try { @@ -109,7 +109,7 @@ public void updateCache(final InMemoryDBExtractionNamespace extractionNamespace, } @Override - public byte[] getCacheValue(final InMemoryDBExtractionNamespace extractionNamespace, final Map cache, final String key, String valueColumn, final Optional decodeConfigOptional) { + public byte[] getCacheValue(final RocksDBExtractionNamespace extractionNamespace, final Map cache, final String key, String valueColumn, final Optional decodeConfigOptional) { try { if (!extractionNamespace.isCacheEnabled()) { @@ -136,7 +136,7 @@ public byte[] getCacheValue(final InMemoryDBExtractionNamespace extractionNamesp } @Override - public String getCacheSize(final InMemoryDBExtractionNamespace extractionNamespace, final Map cache) { + public String getCacheSize(final RocksDBExtractionNamespace extractionNamespace, final Map cache) { if (!extractionNamespace.isCacheEnabled()) { return String.valueOf(lookupService.getSize()); } @@ -153,7 +153,7 @@ public String getCacheSize(final InMemoryDBExtractionNamespace extractionNamespa } @Override - public Long getLastUpdatedTime(final InMemoryDBExtractionNamespace extractionNamespace) { + public Long getLastUpdatedTime(final RocksDBExtractionNamespace extractionNamespace) { if (!extractionNamespace.isCacheEnabled()) { return lookupService.getLastUpdatedTime(new LookupService.LookupData(extractionNamespace)); } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java index d968a9f24..3e58e69a2 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java @@ -7,13 +7,12 @@ import com.google.common.base.StandardSystemProperty; import com.google.common.base.Strings; import com.google.inject.Inject; -import com.google.inject.name.Named; import com.metamx.common.lifecycle.LifecycleStart; import com.metamx.common.lifecycle.LifecycleStop; import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import io.druid.guice.ManageLifecycle; import org.apache.commons.io.FileUtils; import org.apache.commons.io.FilenameUtils; @@ -70,7 +69,7 @@ public RocksDBManager(final MahaNamespaceExtractionConfig mahaNamespaceExtractio this.fileSystem = FileSystem.get(config); } - public String createDB(final InMemoryDBExtractionNamespace extractionNamespace, + public String createDB(final RocksDBExtractionNamespace extractionNamespace, final String lastVersion) throws RocksDBException, IOException { String loadTime = LocalDateTime.now().minus(1, ChronoUnit.DAYS) @@ -150,7 +149,7 @@ public String createDB(final InMemoryDBExtractionNamespace extractionNamespace, return startNewInstance(extractionNamespace, loadTime, hdfsPath, localZippedFileNameWithPath, localPath); } - private String useSnapshotInstance(final InMemoryDBExtractionNamespace extractionNamespace, + private String useSnapshotInstance(final RocksDBExtractionNamespace extractionNamespace, final String loadTime, final String localPath, final File snapShotFile) throws IOException, RocksDBException { @@ -169,7 +168,7 @@ private String useSnapshotInstance(final InMemoryDBExtractionNamespace extractio return loadTime; } - private String startNewInstance(final InMemoryDBExtractionNamespace extractionNamespace, + private String startNewInstance(final RocksDBExtractionNamespace extractionNamespace, final String loadTime, final String hdfsPath, final String localZippedFileNameWithPath, @@ -309,7 +308,7 @@ private void unzipFile(final String localZippedFileNameWithPath) throws IOExcept } private void lookupAuditing(final String localZippedFileNameWithPath, - final InMemoryDBExtractionNamespace extractionNamespace, final String loadTime, + final RocksDBExtractionNamespace extractionNamespace, final String loadTime, long sleepTime, int retryCount) { final String successMarkerPath = String.format("%s/load_time=%s/_SUCCESS", @@ -365,7 +364,7 @@ private void lookupAuditing(final String localZippedFileNameWithPath, } } - private void uploadFileForAuditing(InMemoryDBExtractionNamespace extractionNamespace, + private void uploadFileForAuditing(RocksDBExtractionNamespace extractionNamespace, String loadTime, String successMarkerPath, String localFileNameWithPath) throws IOException { diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java index 7d926b024..c3dc1b17e 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java @@ -12,17 +12,16 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; -import com.yahoo.maha.maha_druid_lookups.query.lookup.InMemoryDBLookupExtractor; +import com.yahoo.maha.maha_druid_lookups.query.lookup.RocksDBLookupExtractor; import com.yahoo.maha.maha_druid_lookups.query.lookup.JDBCLookupExtractor; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.KafkaManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.RocksDBManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; -import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.lookup.LookupExtractor; import javax.annotation.concurrent.GuardedBy; @@ -414,8 +413,8 @@ public void run() private LookupExtractor getLookupExtractor(final ExtractionNamespace extractionNamespace, Map map) { if(extractionNamespace instanceof JDBCExtractionNamespace) { return new JDBCLookupExtractor((JDBCExtractionNamespace)extractionNamespace, map, lookupService); - } else if(extractionNamespace instanceof InMemoryDBExtractionNamespace) { - return new InMemoryDBLookupExtractor((InMemoryDBExtractionNamespace) extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, protobufSchemaFactory, serviceEmitter); + } else if(extractionNamespace instanceof RocksDBExtractionNamespace) { + return new RocksDBLookupExtractor((RocksDBExtractionNamespace) extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, protobufSchemaFactory, serviceEmitter); } else { // return new MapLookupExtractor(map, false); return null; diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupManagerTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupManagerTest.java index 21463ea58..a9fbda3c1 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupManagerTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/missing/lookup/MissingLookupManagerTest.java @@ -4,10 +4,8 @@ import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MissingLookupConfig; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.PasswordProvider; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.TestPasswordProvider; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.TestProtobufSchemaFactory; import io.druid.metadata.MetadataStorageConnectorConfig; @@ -36,21 +34,22 @@ public class MissingLookupManagerTest { @Test public void testMissingLookupManagerTest() throws Exception { - MissingLookupInMemoryDBExtractionNamespaceFactory mlenf = mock(MissingLookupInMemoryDBExtractionNamespaceFactory.class); + MissingLookupRocksDBExtractionNamespaceFactory mlenf = mock( + MissingLookupRocksDBExtractionNamespaceFactory.class); DBI dbi = mock(DBI.class); when(mlenf.ensureDBI(any(), any())).thenReturn(dbi); when(mlenf.ensureKafkaProducer(any())).thenReturn(mock(KafkaProducer.class)); doCallRealMethod().when(mlenf).process(any(), any(), any(), any(), any(), anyString()); TestProtobufSchemaFactory protobufSchemaFactory = new TestProtobufSchemaFactory(); TestPasswordProvider passwordProvider = new TestPasswordProvider(); - MissingLookupManager mlm = spy(new MissingLookupManager<>()); KafkaConsumer kafkaConsumer = mock(KafkaConsumer.class); List> records = new ArrayList<>(); MetadataStorageConnectorConfig metadataStorageConnectorConfig = objectMapper.readValue("{ \"createTables\": false,\"connectURI\": \"jdbc:oracle:thin:@cbrptprod_bf1\",\"user\": \"na_reporting_ws\",\"password\":\"na_reporting_ws.db.prod.pwd\"}", MetadataStorageConnectorConfig.class); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", new MissingLookupConfig(metadataStorageConnectorConfig, "na_reporting.ad", "id", "missing_ad_lookup_topic") ); byte[] byteArray = objectMapper.writeValueAsBytes(extractionNamespace); diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/InMemoryDBLookupExtractorTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractorTest.java similarity index 76% rename from druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/InMemoryDBLookupExtractorTest.java rename to druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractorTest.java index 399d19510..8c7e10fb8 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/InMemoryDBLookupExtractorTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractorTest.java @@ -8,8 +8,7 @@ import com.google.protobuf.Message; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceEventBuilder; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MissingLookupConfig; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.KafkaManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; @@ -25,7 +24,6 @@ import org.testng.annotations.Test; import java.io.File; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -34,7 +32,7 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.*; -public class InMemoryDBLookupExtractorTest { +public class RocksDBLookupExtractorTest { private static final ObjectMapper objectMapper = new ObjectMapper().configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); @Test @@ -43,12 +41,12 @@ public void testBuildWhenDimensionValueIsEmpty() throws Exception { RocksDBManager rocksDBManager = mock(RocksDBManager.class); KafkaManager kafkaManager = mock(KafkaManager.class); ServiceEmitter serviceEmitter = mock(ServiceEmitter.class); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); Map map = new HashMap<>(); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); - String lookupValue = InMemoryDBLookupExtractor.apply(""); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + String lookupValue = RocksDBLookupExtractor.apply(""); Assert.assertNull(lookupValue); } @@ -58,12 +56,12 @@ public void testBuildWhenDimensionValueIsNull() throws Exception { RocksDBManager rocksDBManager = mock(RocksDBManager.class); KafkaManager kafkaManager = mock(KafkaManager.class); ServiceEmitter serviceEmitter = mock(ServiceEmitter.class); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); Map map = new HashMap<>(); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); - String lookupValue = InMemoryDBLookupExtractor.apply(null); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + String lookupValue = RocksDBLookupExtractor.apply(null); Assert.assertNull(lookupValue); } @@ -78,18 +76,18 @@ public void testBuildWhenCacheValueIsNull() throws Exception { when(rocksDBManager.getDB(anyString())).thenReturn(db); MissingLookupConfig missingLookupConfig = new MissingLookupConfig(new MetadataStorageConnectorConfig(), "", "", "some_missing_topic"); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", missingLookupConfig ); Map map = new HashMap<>(); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); mahaLookupQueryElement1.setDimension("abc"); mahaLookupQueryElement1.setValueColumn("status"); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); verify(kafkaManager, times(1)).handleMissingLookup(any(byte[].class), anyString(), anyString()); Assert.assertNull(lookupValue); - InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); verify(kafkaManager, times(1)).handleMissingLookup(any(byte[].class), anyString(), anyString()); verify(serviceEmitter, times(1)).emit(any(ServiceEventBuilder.class)); } @@ -104,15 +102,15 @@ public void handleMissingLookupShouldNotBeCalledWhenNotConfigured() throws Excep ServiceEmitter serviceEmitter = mock(ServiceEmitter.class); when(rocksDBManager.getDB(anyString())).thenReturn(db); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); Map map = new HashMap<>(); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); mahaLookupQueryElement1.setDimension("abc"); mahaLookupQueryElement1.setValueColumn("status"); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); verify(kafkaManager, times(0)).handleMissingLookup(any(byte[].class), anyString(), anyString()); Assert.assertNull(lookupValue); verify(serviceEmitter, times(0)).emit(any(ServiceEventBuilder.class)); @@ -145,21 +143,21 @@ public void testBuildWhenCacheValueIsNotNull() throws Exception{ when(rocksDBManager.getDB(anyString())).thenReturn(db); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); Map map = new HashMap<>(); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); mahaLookupQueryElement1.setDimension("32309719080"); mahaLookupQueryElement1.setValueColumn("status"); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); Assert.assertEquals(lookupValue, "ON"); MahaLookupQueryElement mahaLookupQueryElement2 = new MahaLookupQueryElement(); mahaLookupQueryElement2.setDimension("32309719080"); mahaLookupQueryElement2.setValueColumn("title"); - lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); + lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); Assert.assertEquals(lookupValue, "some title"); } finally { if(db != null) { @@ -199,15 +197,15 @@ public void testBuildWhenInvalidValueColumn() throws Exception { when(rocksDBManager.getDB(anyString())).thenReturn(db); MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); - InMemoryDBExtractionNamespace extractionNamespace = - new InMemoryDBExtractionNamespace("ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null); + RocksDBExtractionNamespace extractionNamespace = + new RocksDBExtractionNamespace("ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); mahaLookupQueryElement1.setDimension("12345"); mahaLookupQueryElement1.setValueColumn("booking_country"); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); Assert.assertNull(lookupValue); } finally { if(db != null) { @@ -227,12 +225,12 @@ public void testDimensionOverrideMap() throws Exception { KafkaManager kafkaManager = mock(KafkaManager.class); ServiceEmitter serviceEmitter = mock(ServiceEmitter.class); MetadataStorageConnectorConfig metadataStorageConnectorConfig = objectMapper.readValue("{ \"createTables\": false,\"connectURI\": \"jdbc:oracle:thin:@cbrptprod_bf1\",\"user\": \"na_reporting_ws\",\"password\":\"na_reporting_ws.db.prod.pwd\"}", MetadataStorageConnectorConfig.class); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", new MissingLookupConfig(metadataStorageConnectorConfig, "na_reporting.ad", "id", "missing_ad_lookup_topic") ); byte[] b = objectMapper.writeValueAsBytes(extractionNamespace); Map map = new HashMap<>(); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); Map dimensionOverrideMap = new HashMap<>(); dimensionOverrideMap.put("12345", "something-12345"); dimensionOverrideMap.put("6789", "something-6789"); @@ -242,14 +240,14 @@ public void testDimensionOverrideMap() throws Exception { mahaLookupQueryElement1.setDimension("12345"); mahaLookupQueryElement1.setValueColumn("name"); mahaLookupQueryElement1.setDimensionOverrideMap(dimensionOverrideMap); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); Assert.assertEquals(lookupValue, "something-12345"); MahaLookupQueryElement mahaLookupQueryElement2 = new MahaLookupQueryElement(); mahaLookupQueryElement2.setDimension(""); mahaLookupQueryElement2.setValueColumn("name"); mahaLookupQueryElement2.setDimensionOverrideMap(dimensionOverrideMap); - lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); + lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); Assert.assertEquals(lookupValue, "Unknown"); } @@ -281,11 +279,11 @@ public void testDecodeConfig() throws Exception { when(rocksDBManager.getDB(anyString())).thenReturn(db); MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); - InMemoryDBExtractionNamespace extractionNamespace = - new InMemoryDBExtractionNamespace("ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", new MissingLookupConfig(metadataStorageConnectorConfig, "na_reporting.ad", "id", "missing_ad_lookup_topic")); + RocksDBExtractionNamespace extractionNamespace = + new RocksDBExtractionNamespace("ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", new MissingLookupConfig(metadataStorageConnectorConfig, "na_reporting.ad", "id", "missing_ad_lookup_topic")); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); Map dimensionOverrideMap = new HashMap<>(); dimensionOverrideMap.put("123", "something-123"); @@ -302,7 +300,7 @@ public void testDecodeConfig() throws Exception { decodeConfig1.setColumnIfValueMatched("last_updated"); decodeConfig1.setColumnIfValueNotMatched("status"); mahaLookupQueryElement1.setDecodeConfig(decodeConfig1); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); Assert.assertEquals(lookupValue, "1470733203505"); MahaLookupQueryElement mahaLookupQueryElement2 = new MahaLookupQueryElement(); @@ -315,7 +313,7 @@ public void testDecodeConfig() throws Exception { decodeConfig2.setColumnIfValueMatched("last_updated"); decodeConfig2.setColumnIfValueNotMatched("status"); mahaLookupQueryElement2.setDecodeConfig(decodeConfig2); - lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); + lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); Assert.assertEquals(lookupValue, "ON"); } finally { if(db != null) { @@ -356,11 +354,11 @@ public void testThrowingDecodeConfig() throws Exception { when(rocksDBManager.getDB(anyString())).thenReturn(db); MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); - InMemoryDBExtractionNamespace extractionNamespace = - new InMemoryDBExtractionNamespace("ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", new MissingLookupConfig(metadataStorageConnectorConfig, "na_reporting.ad", "id", "missing_ad_lookup_topic")); + RocksDBExtractionNamespace extractionNamespace = + new RocksDBExtractionNamespace("ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", new MissingLookupConfig(metadataStorageConnectorConfig, "na_reporting.ad", "id", "missing_ad_lookup_topic")); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); - InMemoryDBLookupExtractor InMemoryDBLookupExtractor = new InMemoryDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); + RocksDBLookupExtractor RocksDBLookupExtractor = new RocksDBLookupExtractor(extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, new TestProtobufSchemaFactory(), serviceEmitter); Map dimensionOverrideMap = new HashMap<>(); dimensionOverrideMap.put("123", "something-123"); @@ -377,7 +375,7 @@ public void testThrowingDecodeConfig() throws Exception { decodeConfig3.setColumnIfValueMatched("fake_column"); decodeConfig3.setColumnIfValueNotMatched("status"); mahaLookupQueryElement3.setDecodeConfig(decodeConfig3); - String lookupValue = InMemoryDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement3)); + String lookupValue = RocksDBLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement3)); Assert.assertEquals(lookupValue, null); } finally { if(db != null) { diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/InMemoryDBExtractionNamespaceCacheFactoryTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactoryTest.java similarity index 93% rename from druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/InMemoryDBExtractionNamespaceCacheFactoryTest.java rename to druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactoryTest.java index 8e2e71ab6..65642f9f6 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/InMemoryDBExtractionNamespaceCacheFactoryTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactoryTest.java @@ -7,7 +7,7 @@ import com.google.protobuf.Message; import com.google.protobuf.Parser; import com.metamx.emitter.service.ServiceEmitter; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.InMemoryDBExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.AdProtos; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.TestProtobufSchemaFactory; import org.apache.commons.io.FileUtils; @@ -29,11 +29,11 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.when; -public class InMemoryDBExtractionNamespaceCacheFactoryTest { +public class RocksDBExtractionNamespaceCacheFactoryTest { @InjectMocks - InMemoryDBExtractionNamespaceCacheFactory obj = - new InMemoryDBExtractionNamespaceCacheFactory(); + RocksDBExtractionNamespaceCacheFactory obj = + new RocksDBExtractionNamespaceCacheFactory(); @Mock RocksDBManager rocksDBManager; @@ -72,7 +72,7 @@ public void testUpdateCacheWithGreaterLastUpdated() throws Exception{ when(rocksDBManager.getDB(anyString())).thenReturn(db); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); @@ -133,7 +133,7 @@ public void testUpdateCacheWithLesserLastUpdated() throws Exception{ when(rocksDBManager.getDB(anyString())).thenReturn(db); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); @@ -194,7 +194,7 @@ public void testGetCacheValue() throws Exception{ when(rocksDBManager.getDB(anyString())).thenReturn(db); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); @@ -226,7 +226,7 @@ public void testGetCacheValueWhenNull() throws Exception{ when(rocksDBManager.getDB(anyString())).thenReturn(db); - InMemoryDBExtractionNamespace extractionNamespace = new InMemoryDBExtractionNamespace( + RocksDBExtractionNamespace extractionNamespace = new RocksDBExtractionNamespace( "ad_lookup", "blah", "blah", new Period(), "", true, false, "ad_lookup", "last_updated", null ); From b1fb425ef8d3c5f151d617ffcc160dc16a9ab038 Mon Sep 17 00:00:00 2001 From: patelh Date: Wed, 20 Feb 2019 18:14:05 -0800 Subject: [PATCH 02/15] Add mongo extraction namespace --- druid-lookups/pom.xml | 17 + .../query/lookup/DecodeConfig.java | 33 ++ .../query/lookup/JDBCLookupExtractor.java | 181 +---------- .../query/lookup/MahaLookupExtractionFn.java | 63 ++-- .../query/lookup/MahaLookupExtractor.java | 41 +++ .../lookup/MahaLookupExtractorFactory.java | 8 +- .../lookup/MahaLookupIntrospectHandler.java | 6 +- .../MahaRegisteredLookupExtractionFn.java | 172 +++++----- .../query/lookup/MongoLookupExtractor.java | 25 ++ .../OnlineDatastoreLookupExtractor.java | 159 +++++++++ .../query/lookup/RocksDBLookupExtractor.java | 72 ++--- .../lookup/namespace/ExtractionNamespace.java | 4 + .../namespace/JDBCExtractionNamespace.java | 135 +++----- .../namespace/MongoExtractionNamespace.java | 195 +++++++++++ .../MongoStorageConnectorConfig.java | 202 ++++++++++++ .../OnlineDatastoreExtractionNamespace.java | 20 ++ .../namespace/RocksDBExtractionNamespace.java | 60 ++-- .../JDBCExtractionNamespaceCacheFactory.java | 48 +-- .../server/lookup/namespace/KafkaManager.java | 6 +- .../MahaNamespaceExtractionConfig.java | 15 +- .../MahaNamespaceExtractionModule.java | 15 +- .../MahaNamespacesCacheResource.java | 26 +- .../MongoExtractionNamespaceCacheFactory.java | 306 ++++++++++++++++++ .../namespace/NoopAuthHeaderFactory.java | 11 + ... MahaNamespaceExtractionCacheManager.java} | 123 +++---- ...pMahaNamespaceExtractionCacheManager.java} | 6 +- .../FlatMultiValueDocumentProcessor.java | 115 +++++++ .../namespace/entity/LookupBuilder.java | 17 + .../entity/MongoDocumentProcessor.java | 23 ++ .../io.druid.initialization.DruidModule | 2 +- .../query/lookup/JDBCLookupExtractorTest.java | 18 +- .../MahaRegisteredLookupExtractionFnTest.java | 25 +- .../lookup/MongoLookupExtractorTest.java | 230 +++++++++++++ .../MongoExtractionNamespaceTest.java | 63 ++++ .../MongoStorageConnectorConfigTest.java | 88 +++++ .../MahaNamespaceExtractionModuleTest.java | 51 +++ .../MahaNamespacesCacheResourceTest.java | 6 +- ...goExtractionNamespaceCacheFactoryTest.java | 228 +++++++++++++ .../src/test/resources/mongo_advertiser.json | 29 ++ .../resources/mongo_advertiser_addition.json | 38 +++ .../resources/mongo_advertiser_update.json | 29 ++ .../resources/mongo_extraction_namespace.json | 31 ++ pom.xml | 1 + 43 files changed, 2320 insertions(+), 623 deletions(-) create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractor.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractor.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/OnlineDatastoreLookupExtractor.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespace.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfig.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/OnlineDatastoreExtractionNamespace.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/NoopAuthHeaderFactory.java rename druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/{MahaExtractionCacheManager.java => MahaNamespaceExtractionCacheManager.java} (88%) rename druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/{OnHeapMahaExtractionCacheManager.java => OnHeapMahaNamespaceExtractionCacheManager.java} (93%) create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/FlatMultiValueDocumentProcessor.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/LookupBuilder.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/MongoDocumentProcessor.java rename druid-lookups/src/{ => main}/resources/META-INF/services/io.druid.initialization.DruidModule (58%) create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractorTest.java create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespaceTest.java create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfigTest.java create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java create mode 100644 druid-lookups/src/test/resources/mongo_advertiser.json create mode 100644 druid-lookups/src/test/resources/mongo_advertiser_addition.json create mode 100644 druid-lookups/src/test/resources/mongo_advertiser_update.json create mode 100644 druid-lookups/src/test/resources/mongo_extraction_namespace.json diff --git a/druid-lookups/pom.xml b/druid-lookups/pom.xml index 6df644bbd..1cc6d7ce2 100644 --- a/druid-lookups/pom.xml +++ b/druid-lookups/pom.xml @@ -196,7 +196,24 @@ 1.9 jar + + org.mongodb + mongo-java-driver + ${mongo-java-driver.version} + + + de.bwaldvogel + mongo-java-server + 1.5.0 + test + + + io.netty + * + + + junit junit diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/DecodeConfig.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/DecodeConfig.java index 4c126f1aa..2dcbb4140 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/DecodeConfig.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/DecodeConfig.java @@ -2,8 +2,14 @@ // Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. package com.yahoo.maha.maha_druid_lookups.query.lookup; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Throwables; +import io.druid.query.extraction.ExtractionCacheHelper; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Objects; public class DecodeConfig { @@ -52,6 +58,33 @@ public void setColumnIfValueNotMatched(String columnIfValueNotMatched) { this.columnIfValueNotMatched = columnIfValueNotMatched; } + @JsonIgnore + public byte[] getCacheKey() { + try { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + if (getColumnToCheck() != null) { + outputStream.write(getColumnToCheck().getBytes()); + } + outputStream.write(0xFF); + if (getValueToCheck() != null) { + outputStream.write(getValueToCheck().getBytes()); + } + outputStream.write(0xFF); + if (getColumnIfValueMatched() != null) { + outputStream.write(getColumnIfValueMatched().getBytes()); + } + outputStream.write(0xFF); + if (getColumnIfValueNotMatched() != null) { + outputStream.write(getColumnIfValueNotMatched().getBytes()); + } + outputStream.write(0xFF); + return outputStream.toByteArray(); + } catch (IOException ex) { + // If ByteArrayOutputStream.write has problems, that is a very bad thing + throw Throwables.propagate(ex); + } + } + @Override public String toString() { return "DecodeConfig{" + diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractor.java index ffb1ae13c..3bff51083 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractor.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractor.java @@ -2,190 +2,23 @@ // Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. package com.yahoo.maha.maha_druid_lookups.query.lookup; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.base.Strings; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; -import io.druid.query.lookup.LookupExtractor; -import javax.annotation.Nullable; -import javax.validation.constraints.NotNull; -import java.io.ByteArrayOutputStream; -import java.io.IOException; +import java.lang.invoke.MethodHandles; import java.util.List; import java.util.Map; -import java.util.Optional; -import static java.nio.charset.StandardCharsets.UTF_8; +public class JDBCLookupExtractor> extends OnlineDatastoreLookupExtractor { + private static final Logger LOG = new Logger(MethodHandles.lookup().lookupClass()); -public class JDBCLookupExtractor> extends LookupExtractor -{ - private static final Logger LOG = new Logger(JDBCLookupExtractor.class); - private static final ObjectMapper objectMapper = new ObjectMapper(); - private final Map map; - private final JDBCExtractionNamespace extractionNamespace; - private LookupService lookupService; - private final Map columnIndexMap; - - public JDBCLookupExtractor(JDBCExtractionNamespace extractionNamespace, Map map, LookupService lookupService) - { - this.extractionNamespace = extractionNamespace; - this.map = Preconditions.checkNotNull(map, "map"); - this.lookupService = lookupService; - int index = 0; - ImmutableMap.Builder builder = ImmutableMap.builder(); - for(String col : extractionNamespace.getColumnList()) { - builder.put(col, index); - index += 1; - } - this.columnIndexMap = builder.build(); - } - - public Map getMap() - { - return ImmutableMap.copyOf(map); - } - - @Nullable - @Override - public String apply(@NotNull String val) - { - - try { - - if("".equals(Strings.nullToEmpty(val))) { - return null; - } - - MahaLookupQueryElement mahaLookupQueryElement = objectMapper.readValue(val, MahaLookupQueryElement.class); - String dimension = Strings.nullToEmpty(mahaLookupQueryElement.getDimension()); - String valueColumn = mahaLookupQueryElement.getValueColumn(); - DecodeConfig decodeConfig = mahaLookupQueryElement.getDecodeConfig(); - Map dimensionOverrideMap = mahaLookupQueryElement.getDimensionOverrideMap(); - - if(dimensionOverrideMap != null && dimensionOverrideMap.containsKey(dimension)) { - return Strings.emptyToNull(dimensionOverrideMap.get(dimension)); - } - - if (!extractionNamespace.isCacheEnabled()) { - Optional decodeConfigOptional = (decodeConfig == null) ? Optional.empty() : Optional.of(decodeConfig); - byte[] cacheByteValue = lookupService.lookup(new LookupService.LookupData(extractionNamespace, - dimension, valueColumn, decodeConfigOptional)); - return (cacheByteValue == null || cacheByteValue.length == 0) ? null : new String(cacheByteValue, UTF_8); - } else { - U cacheValueArray = map.get(dimension); - if(cacheValueArray == null) { - return null; - } - if(decodeConfig != null) { - return handleDecode(cacheValueArray, decodeConfig); - } - else { - int columnIndex = getColumnIndex(extractionNamespace, valueColumn); - if (columnIndex < 0 || columnIndex >= cacheValueArray.size()) { - LOG.error("Invalid columnIndex [%s], cacheValueArray is [%s]", columnIndex, cacheValueArray); - return null; - } - return Strings.emptyToNull(cacheValueArray.get(columnIndex)); - } - } - - } catch(Exception e) { - LOG.error(e, "Exception in JDBCLookupExtractor apply"); - } - return null; - } - - private String handleDecode(U row, DecodeConfig decodeConfig) { - - final int columnToCheckIndex = getColumnIndex(extractionNamespace, decodeConfig.getColumnToCheck()); - if (columnToCheckIndex < 0 || columnToCheckIndex >= row.size() ) { - return null; - } - - final String valueFromColumnToCheck = row.get(columnToCheckIndex); - - if(valueFromColumnToCheck != null && valueFromColumnToCheck.equals(decodeConfig.getValueToCheck())) { - final int columnIfValueMatchedIndex = getColumnIndex(extractionNamespace, decodeConfig.getColumnIfValueMatched()); - if (columnIfValueMatchedIndex < 0) { - return null; - } - return Strings.emptyToNull(row.get(columnIfValueMatchedIndex)); - } else { - final int columnIfValueNotMatchedIndex = getColumnIndex(extractionNamespace, decodeConfig.getColumnIfValueNotMatched()); - if (columnIfValueNotMatchedIndex < 0) { - return null; - } - return Strings.emptyToNull(row.get(columnIfValueNotMatchedIndex)); - } - } - - private int getColumnIndex(final JDBCExtractionNamespace extractionNamespace, String valueColumn) { - if(columnIndexMap.containsKey(valueColumn)) { - return columnIndexMap.get(valueColumn); - } - return -1; - } - - @Override - public List unapply(final String value) - { - return Lists.newArrayList(Maps.filterKeys(map, new Predicate() - { - @Override public boolean apply(@Nullable String key) - { - return map.get(key).equals(Strings.nullToEmpty(value)); - } - }).keySet()); - - } - - @Override - public byte[] getCacheKey() - { - try { - final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - for (Map.Entry entry : map.entrySet()) { - final String key = entry.getKey(); - if (!Strings.isNullOrEmpty(key)) { - outputStream.write(key.getBytes()); - } - outputStream.write((byte)0xFF); - } - return outputStream.toByteArray(); - } - catch (IOException ex) { - // If ByteArrayOutputStream.write has problems, that is a very bad thing - throw Throwables.propagate(ex); - } - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - JDBCLookupExtractor that = (JDBCLookupExtractor) o; - - return map.equals(that.map); + public JDBCLookupExtractor(JDBCExtractionNamespace extractionNamespace, Map map, LookupService lookupService) { + super(extractionNamespace, map, lookupService); } @Override - public int hashCode() - { - return map.hashCode(); + protected Logger LOGGER() { + return LOG; } - } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFn.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFn.java index 73f086bac..0114b0f46 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFn.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFn.java @@ -14,9 +14,9 @@ import javax.annotation.Nullable; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.Map; -public class MahaLookupExtractionFn extends FunctionalExtraction -{ +public class MahaLookupExtractionFn extends FunctionalExtraction { private final LookupExtractor lookup; private final boolean optimize; // Thes are retained for auto generated hashCode and Equals @@ -24,23 +24,27 @@ public class MahaLookupExtractionFn extends FunctionalExtraction private final String replaceMissingValueWith; private final boolean injective; - @JsonCreator public MahaLookupExtractionFn( - @JsonProperty("lookup") final LookupExtractor lookup, - @JsonProperty("retainMissingValue") final boolean retainMissingValue, - @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, - @JsonProperty("injective") final boolean injective, - @JsonProperty("optimize") Boolean optimize - ) - { + final LookupExtractor lookup + , final boolean retainMissingValue + , final String replaceMissingValueWith + , final boolean injective + , Boolean optimize + , String valueColumn + , DecodeConfig decodeConfig + , Map dimensionOverrideMap + ) { super( - new Function() - { + new Function() { @Nullable @Override - public String apply(String input) - { - return lookup.apply(Strings.nullToEmpty(input)); + public String apply(String input) { + if (lookup instanceof MahaLookupExtractor) { + MahaLookupExtractor mahaLookupExtractor = (MahaLookupExtractor) lookup; + return mahaLookupExtractor.apply(Strings.nullToEmpty(input), valueColumn, decodeConfig, dimensionOverrideMap); + } else { + return lookup.apply(Strings.nullToEmpty(input)); + } } }, retainMissingValue, @@ -56,35 +60,35 @@ public String apply(String input) @JsonProperty - public LookupExtractor getLookup() - { + public LookupExtractor getLookup() { return lookup; } @Override @JsonProperty - public boolean isRetainMissingValue() {return super.isRetainMissingValue();} + public boolean isRetainMissingValue() { + return super.isRetainMissingValue(); + } @Override @JsonProperty - public String getReplaceMissingValueWith() {return super.getReplaceMissingValueWith();} + public String getReplaceMissingValueWith() { + return super.getReplaceMissingValueWith(); + } @Override @JsonProperty - public boolean isInjective() - { + public boolean isInjective() { return super.isInjective(); } @JsonProperty("optimize") - public boolean isOptimize() - { + public boolean isOptimize() { return optimize; } @Override - public byte[] getCacheKey() - { + public byte[] getCacheKey() { try { final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); outputStream.write(ExtractionCacheHelper.CACHE_TYPE_ID_LOOKUP); @@ -97,16 +101,14 @@ public byte[] getCacheKey() outputStream.write(isRetainMissingValue() ? 1 : 0); outputStream.write(isOptimize() ? 1 : 0); return outputStream.toByteArray(); - } - catch (IOException ex) { + } catch (IOException ex) { // If ByteArrayOutputStream.write has problems, that is a very bad thing throw Throwables.propagate(ex); } } @Override - public boolean equals(Object o) - { + public boolean equals(Object o) { if (this == o) { return true; } @@ -135,8 +137,7 @@ public boolean equals(Object o) } @Override - public int hashCode() - { + public int hashCode() { int result = getLookup() != null ? getLookup().hashCode() : 0; result = 31 * result + (isOptimize() ? 1 : 0); result = 31 * result + (isRetainMissingValue() ? 1 : 0); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractor.java new file mode 100644 index 000000000..04d8595cd --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractor.java @@ -0,0 +1,41 @@ +package com.yahoo.maha.maha_druid_lookups.query.lookup; + +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import io.druid.query.lookup.LookupExtractor; +import com.metamx.common.logger.Logger; + +import java.util.Map; + +abstract public class MahaLookupExtractor extends LookupExtractor { + private static final Logger LOG = new Logger(JDBCLookupExtractor.class); + + private static final ObjectMapper objectMapper = new ObjectMapper(); + + @Nullable + public abstract String apply(@NotNull String key, @NotNull String valueColumn, DecodeConfig decodeConfig, Map dimensionOverrideMap); + + @Nullable + @Override + public String apply(@NotNull String key) { + try { + if (key == null) { + return null; + } + + MahaLookupQueryElement mahaLookupQueryElement = objectMapper.readValue(key, MahaLookupQueryElement.class); + String dimension = Strings.nullToEmpty(mahaLookupQueryElement.getDimension()); + String valueColumn = mahaLookupQueryElement.getValueColumn(); + DecodeConfig decodeConfig = mahaLookupQueryElement.getDecodeConfig(); + Map dimensionOverrideMap = mahaLookupQueryElement.getDimensionOverrideMap(); + + return apply(dimension, valueColumn, decodeConfig, dimensionOverrideMap); + } catch (Exception e) { + LOG.error(e, "Exception in MahaLookupExtractor apply"); + } + return null; + } +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java index c1ec79ceb..48cd1736f 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java @@ -13,7 +13,7 @@ import com.metamx.common.StringUtils; import com.metamx.common.logger.Logger; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; import io.druid.query.lookup.LookupIntrospectHandler; @@ -39,7 +39,7 @@ public class MahaLookupExtractorFactory implements LookupExtractorFactory private volatile boolean started = false; private final ReadWriteLock startStopSync = new ReentrantReadWriteLock(); - private final MahaExtractionCacheManager manager; + private final MahaNamespaceExtractionCacheManager manager; private final LookupIntrospectHandler lookupIntrospectHandler; private final ExtractionNamespace extractionNamespace; private final long firstCacheTimeout; @@ -52,7 +52,7 @@ public MahaLookupExtractorFactory( @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, @JsonProperty("firstCacheTimeout") long firstCacheTimeout, @JsonProperty("injective") boolean injective, - @JacksonInject final MahaExtractionCacheManager manager + @JacksonInject final MahaNamespaceExtractionCacheManager manager ) { this.extractionNamespace = Preconditions.checkNotNull( @@ -70,7 +70,7 @@ public MahaLookupExtractorFactory( @VisibleForTesting public MahaLookupExtractorFactory( ExtractionNamespace extractionNamespace, - MahaExtractionCacheManager manager + MahaNamespaceExtractionCacheManager manager ) { this(extractionNamespace, 60000, false, manager); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupIntrospectHandler.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupIntrospectHandler.java index 453f26e04..32f9d7be7 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupIntrospectHandler.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupIntrospectHandler.java @@ -4,7 +4,7 @@ import com.google.common.collect.ImmutableMap; import com.metamx.common.ISE; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; import io.druid.common.utils.ServletResourceUtils; import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.lookup.LookupIntrospectHandler; @@ -20,10 +20,10 @@ public class MahaLookupIntrospectHandler implements LookupIntrospectHandler { private final MahaLookupExtractorFactory factory; private final String extractorID; - private final MahaExtractionCacheManager manager; + private final MahaNamespaceExtractionCacheManager manager; public MahaLookupIntrospectHandler( MahaLookupExtractorFactory factory, - MahaExtractionCacheManager manager, + MahaNamespaceExtractionCacheManager manager, String extractorID ) { this.factory = factory; diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java index 8c6340aa7..69929eee7 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java @@ -11,11 +11,14 @@ import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.metamx.common.logger.Logger; import io.druid.query.extraction.ExtractionFn; import io.druid.query.lookup.LookupReferencesManager; import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -23,8 +26,7 @@ import com.google.common.base.Strings; @JsonTypeName("mahaRegisteredLookup") -public class MahaRegisteredLookupExtractionFn implements ExtractionFn -{ +public class MahaRegisteredLookupExtractionFn implements ExtractionFn { private static final Logger LOG = new Logger(MahaRegisteredLookupExtractionFn.class); // Protected for moving to not-null by `delegateLock` private volatile MahaLookupExtractionFn delegate = null; @@ -56,8 +58,7 @@ public MahaRegisteredLookupExtractionFn( @Nullable @JsonProperty("decode") DecodeConfig decodeConfig, @Nullable @JsonProperty("dimensionOverrideMap") Map dimensionOverrideMap, @Nullable @JsonProperty("useQueryLevelCache") Boolean useQueryLevelCache - ) - { + ) { Preconditions.checkArgument(lookup != null, "`lookup` required"); this.manager = manager; this.objectMapper = objectMapper; @@ -73,127 +74,141 @@ public MahaRegisteredLookupExtractionFn( } @JsonProperty("lookup") - public String getLookup() - { + public String getLookup() { return lookup; } @JsonProperty("retainMissingValue") - public boolean isRetainMissingValue() - { + public boolean isRetainMissingValue() { return retainMissingValue; } @JsonProperty("replaceMissingValueWith") - public String getReplaceMissingValueWith() - { + public String getReplaceMissingValueWith() { return replaceMissingValueWith; } @JsonProperty("injective") - public boolean isInjective() - { + public boolean isInjective() { return injective; } @JsonProperty("optimize") - public boolean isOptimize() - { + public boolean isOptimize() { return optimize; } @JsonProperty("valueColumn") - public String getValueColumn() - { + public String getValueColumn() { return valueColumn; } @JsonProperty("decode") - public DecodeConfig getDecodeConfig() - { + public DecodeConfig getDecodeConfig() { return decodeConfig; } @JsonProperty("dimensionOverrideMap") - public Map getDimensionOverrideMap() - { + public Map getDimensionOverrideMap() { return dimensionOverrideMap; } @JsonProperty("useQueryLevelCache") - public boolean isUseQueryLevelCache() - { + public boolean isUseQueryLevelCache() { return useQueryLevelCache; } @Override - public byte[] getCacheKey() - { - final byte[] keyPrefix = getClass().getCanonicalName().getBytes(); - final byte[] lookupName = getLookup().getBytes(); - final byte[] delegateKey = ensureDelegate().getCacheKey(); - return ByteBuffer - .allocate(keyPrefix.length + 1 + lookupName.length + 1 + delegateKey.length) - .put(keyPrefix).put((byte) 0xFF) - .put(lookupName).put((byte) 0xFF) - .put(delegateKey) - .array(); + public byte[] getCacheKey() { + try { + final byte[] keyPrefix = getClass().getCanonicalName().getBytes(); + final byte[] lookupBytes = getLookup().getBytes(); + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + outputStream.write(keyPrefix); + outputStream.write(0xFF); + outputStream.write(lookupBytes); + if (getValueColumn() != null) { + outputStream.write(getValueColumn().getBytes()); + } + outputStream.write(0xFF); + if (getDecodeConfig() != null) { + outputStream.write(getDecodeConfig().getCacheKey()); + } + outputStream.write(0xFF); + if (getDimensionOverrideMap() != null) { + outputStream.write(getDimensionOverrideMap().hashCode()); + } + outputStream.write(0xFF); + outputStream.write(isUseQueryLevelCache() ? 1 : 0); + outputStream.write(0xFF); + outputStream.write(ensureDelegate().getCacheKey()); + return outputStream.toByteArray(); + } catch (IOException ex) { + // If ByteArrayOutputStream.write has problems, that is a very bad thing + throw Throwables.propagate(ex); + } } @Override - public String apply(Object value) - { + public String apply(Object value) { return ensureDelegate().apply(value); } @Override - public String apply(String value) - { - if("".equals(Strings.nullToEmpty(value))) + public String apply(String value) { + if ("".equals(Strings.nullToEmpty(value))) return null; - String serializedElement = isUseQueryLevelCache() ? - ensureCache().get(value, key -> getSerializedLookupQueryElement(value)) : - getSerializedLookupQueryElement(value); - return ensureDelegate().apply(serializedElement); - } - - String getSerializedLookupQueryElement(String value) { - String serializedElement = ""; - try { - MahaLookupQueryElement mahaLookupQueryElement = new MahaLookupQueryElement(); - mahaLookupQueryElement.setDimension(value); - mahaLookupQueryElement.setValueColumn(valueColumn); - mahaLookupQueryElement.setDecodeConfig(decodeConfig); - mahaLookupQueryElement.setDimensionOverrideMap(dimensionOverrideMap); - serializedElement = objectMapper.writeValueAsString(mahaLookupQueryElement); - } catch (JsonProcessingException e) { - LOG.error(e, e.getMessage()); + if (isUseQueryLevelCache()) { + Cache cache = ensureCache(); + String cachedResult = cache.getIfPresent(value); + if (cachedResult != null) { + return cachedResult; + } else { + String result = ensureDelegate().apply(value); + if (result != null) { + cache.put(value, result); + } + return result; + } + } else { + return ensureDelegate().apply(value); } - return serializedElement; } +// String getSerializedLookupQueryElement(String value) { +// String serializedElement = ""; +// try { +// MahaLookupQueryElement mahaLookupQueryElement = new MahaLookupQueryElement(); +// mahaLookupQueryElement.setDimension(value); +// mahaLookupQueryElement.setValueColumn(valueColumn); +// mahaLookupQueryElement.setDecodeConfig(decodeConfig); +// mahaLookupQueryElement.setDimensionOverrideMap(dimensionOverrideMap); +// serializedElement = objectMapper.writeValueAsString(mahaLookupQueryElement); +// } catch (JsonProcessingException e) { +// LOG.error(e, e.getMessage()); +// } +// return serializedElement; +// } + @Override - public String apply(long value) - { + public String apply(long value) { return ensureDelegate().apply(value); } @Override - public boolean preservesOrdering() - { + public boolean preservesOrdering() { return ensureDelegate().preservesOrdering(); } @Override - public ExtractionType getExtractionType() - { + public ExtractionType getExtractionType() { return ensureDelegate().getExtractionType(); } Cache ensureCache() { - if(null == cache) { + if (null == cache) { synchronized (cacheLock) { - if(null == cache) { + if (null == cache) { this.cache = Caffeine .newBuilder() .maximumSize(10_000) @@ -205,18 +220,22 @@ Cache ensureCache() { return cache; } - private MahaLookupExtractionFn ensureDelegate() - { + private MahaLookupExtractionFn ensureDelegate() { if (null == delegate) { // http://www.javamex.com/tutorials/double_checked_locking.shtml synchronized (delegateLock) { if (null == delegate) { delegate = new MahaLookupExtractionFn( - Preconditions.checkNotNull(manager.get(getLookup()), "Lookup [%s] not found", getLookup()).getLookupExtractorFactory().get(), - isRetainMissingValue(), - getReplaceMissingValueWith(), - isInjective(), - isOptimize() + Preconditions.checkNotNull(manager.get(getLookup()) + , "Lookup [%s] not found", getLookup() + ).getLookupExtractorFactory().get() + , isRetainMissingValue() + , getReplaceMissingValueWith() + , isInjective() + , isOptimize() + , valueColumn + , decodeConfig + , dimensionOverrideMap ); } } @@ -225,8 +244,7 @@ private MahaLookupExtractionFn ensureDelegate() } @Override - public boolean equals(Object o) - { + public boolean equals(Object o) { if (this == o) { return true; } @@ -249,7 +267,7 @@ public boolean equals(Object o) return false; } - if(isUseQueryLevelCache() != that.isUseQueryLevelCache()) { + if (isUseQueryLevelCache() != that.isUseQueryLevelCache()) { return false; } @@ -259,8 +277,7 @@ public boolean equals(Object o) } @Override - public int hashCode() - { + public int hashCode() { int result = getLookup().hashCode(); result = 31 * result + (isRetainMissingValue() ? 1 : 0); result = 31 * result + (getReplaceMissingValueWith() != null ? getReplaceMissingValueWith().hashCode() : 0); @@ -271,8 +288,7 @@ public int hashCode() } @Override - public String toString() - { + public String toString() { return "MahaRegisteredLookupExtractionFn{" + "delegate=" + delegate + ", lookup='" + lookup + '\'' + diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractor.java new file mode 100644 index 000000000..0500938f2 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractor.java @@ -0,0 +1,25 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.query.lookup; + +import com.metamx.common.logger.Logger; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; + +import java.lang.invoke.MethodHandles; +import java.util.List; +import java.util.Map; + +public class MongoLookupExtractor> extends OnlineDatastoreLookupExtractor { + private static final Logger LOG = new Logger(MethodHandles.lookup().lookupClass()); + + public MongoLookupExtractor(MongoExtractionNamespace extractionNamespace, Map map, LookupService lookupService) { + super(extractionNamespace, map, lookupService); + } + + @Override + protected Logger LOGGER() { + return LOG; + } + +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/OnlineDatastoreLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/OnlineDatastoreLookupExtractor.java new file mode 100644 index 000000000..5acbd0a7d --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/OnlineDatastoreLookupExtractor.java @@ -0,0 +1,159 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.query.lookup; + +import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; +import com.google.common.base.Strings; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.metamx.common.logger.Logger; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.OnlineDatastoreExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; + +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static java.nio.charset.StandardCharsets.UTF_8; + +abstract public class OnlineDatastoreLookupExtractor> extends MahaLookupExtractor { + private final Map map; + private final OnlineDatastoreExtractionNamespace extractionNamespace; + private LookupService lookupService; + private final ImmutableMap columnIndexMap; + + abstract protected Logger LOGGER(); + + OnlineDatastoreLookupExtractor(OnlineDatastoreExtractionNamespace extractionNamespace, Map map, LookupService lookupService) { + this.extractionNamespace = extractionNamespace; + this.map = Preconditions.checkNotNull(map, "map"); + this.lookupService = lookupService; + this.columnIndexMap = extractionNamespace.getColumnIndexMap(); + } + + public Map getMap() { + return ImmutableMap.copyOf(map); + } + + @Nullable + public String apply(@NotNull String key, @NotNull String valueColumn, DecodeConfig decodeConfig, Map dimensionOverrideMap) { + try { + + if (key == null) { + return null; + } + + if (dimensionOverrideMap != null && dimensionOverrideMap.containsKey(key)) { + return Strings.emptyToNull(dimensionOverrideMap.get(key)); + } + + if (!extractionNamespace.isCacheEnabled()) { + Optional decodeConfigOptional = (decodeConfig == null) ? Optional.empty() : Optional.of(decodeConfig); + byte[] cacheByteValue = lookupService.lookup(new LookupService.LookupData(extractionNamespace, + key, valueColumn, decodeConfigOptional)); + return (cacheByteValue == null || cacheByteValue.length == 0) ? null : new String(cacheByteValue, UTF_8); + } else { + U cacheValueArray = map.get(key); + if (cacheValueArray == null) { + return null; + } + if (decodeConfig != null) { + return handleDecode(cacheValueArray, decodeConfig); + } else { + int columnIndex = getColumnIndex(valueColumn); + if (columnIndex < 0 || columnIndex >= cacheValueArray.size()) { + LOGGER().error("Invalid columnIndex [%s], cacheValueArray is [%s]", columnIndex, cacheValueArray); + return null; + } + return Strings.emptyToNull(cacheValueArray.get(columnIndex)); + } + } + + } catch (Exception e) { + LOGGER().error(e, "Exception in OnlineDatastoreLookupExtractor apply"); + } + return null; + } + + private String handleDecode(U row, DecodeConfig decodeConfig) { + + final int columnToCheckIndex = getColumnIndex(decodeConfig.getColumnToCheck()); + if (columnToCheckIndex < 0 || columnToCheckIndex >= row.size()) { + return null; + } + + final String valueFromColumnToCheck = row.get(columnToCheckIndex); + + if (valueFromColumnToCheck != null && valueFromColumnToCheck.equals(decodeConfig.getValueToCheck())) { + final int columnIfValueMatchedIndex = getColumnIndex(decodeConfig.getColumnIfValueMatched()); + if (columnIfValueMatchedIndex < 0) { + return null; + } + return Strings.emptyToNull(row.get(columnIfValueMatchedIndex)); + } else { + final int columnIfValueNotMatchedIndex = getColumnIndex(decodeConfig.getColumnIfValueNotMatched()); + if (columnIfValueNotMatchedIndex < 0) { + return null; + } + return Strings.emptyToNull(row.get(columnIfValueNotMatchedIndex)); + } + } + + private int getColumnIndex(String valueColumn) { + if (columnIndexMap.containsKey(valueColumn)) { + return columnIndexMap.get(valueColumn); + } + return -1; + } + + @Override + public List unapply(final String value) { + return Lists.newArrayList(Maps.filterKeys(map, new Predicate() { + @Override + public boolean apply(@Nullable String key) { + return map.get(key).equals(Strings.nullToEmpty(value)); + } + }).keySet()); + + } + + @Override + public byte[] getCacheKey() { + try { + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + outputStream.write(extractionNamespace.toString().getBytes()); + outputStream.write((byte) 0xFF); + return outputStream.toByteArray(); + } catch (IOException ex) { + // If ByteArrayOutputStream.write has problems, that is a very bad thing + throw Throwables.propagate(ex); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + OnlineDatastoreLookupExtractor that = (OnlineDatastoreLookupExtractor) o; + + return map.equals(that.map); + } + + @Override + public int hashCode() { + return map.hashCode(); + } + +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java index 3b4887dd7..590b06b2a 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java @@ -39,8 +39,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; -public class RocksDBLookupExtractor extends LookupExtractor -{ +public class RocksDBLookupExtractor extends MahaLookupExtractor { private static final Logger LOG = new Logger(RocksDBLookupExtractor.class); private static final ObjectMapper objectMapper = new ObjectMapper(); private final Map map; @@ -54,8 +53,8 @@ public class RocksDBLookupExtractor extends LookupExtractor private final byte[] extractionNamespaceAsByteArray; public RocksDBLookupExtractor(RocksDBExtractionNamespace extractionNamespace, Map map, - LookupService lookupService, RocksDBManager rocksDBManager, KafkaManager kafkaManager, - ProtobufSchemaFactory protobufSchemaFactory, ServiceEmitter serviceEmitter) { + LookupService lookupService, RocksDBManager rocksDBManager, KafkaManager kafkaManager, + ProtobufSchemaFactory protobufSchemaFactory, ServiceEmitter serviceEmitter) { this.extractionNamespace = extractionNamespace; this.map = Preconditions.checkNotNull(map, "map"); this.rocksDBManager = rocksDBManager; @@ -76,35 +75,26 @@ public RocksDBLookupExtractor(RocksDBExtractionNamespace extractionNamespace, Ma } - public Map getMap() - { + public Map getMap() { return ImmutableMap.copyOf(map); } @Nullable - @Override - public String apply(@NotNull String val) - { + public String apply(@NotNull String key, @NotNull String valueColumn, DecodeConfig decodeConfig, Map dimensionOverrideMap) { try { - if("".equals(Strings.nullToEmpty(val))) { + if (key == null) { return null; } - MahaLookupQueryElement mahaLookupQueryElement = objectMapper.readValue(val, MahaLookupQueryElement.class); - String dimension = Strings.nullToEmpty(mahaLookupQueryElement.getDimension()); - String valueColumn = mahaLookupQueryElement.getValueColumn(); - DecodeConfig decodeConfig = mahaLookupQueryElement.getDecodeConfig(); - Map dimensionOverrideMap = mahaLookupQueryElement.getDimensionOverrideMap(); - - if(dimensionOverrideMap != null && dimensionOverrideMap.containsKey(dimension)) { - return Strings.emptyToNull(dimensionOverrideMap.get(dimension)); + if (dimensionOverrideMap != null && dimensionOverrideMap.containsKey(key)) { + return Strings.emptyToNull(dimensionOverrideMap.get(key)); } if (!extractionNamespace.isCacheEnabled()) { Optional decodeConfigOptional = (decodeConfig == null) ? Optional.empty() : Optional.of(decodeConfig); byte[] cacheByteValue = lookupService.lookup(new LookupService.LookupData(extractionNamespace, - dimension, valueColumn, decodeConfigOptional)); + key, valueColumn, decodeConfigOptional)); return (cacheByteValue == null || cacheByteValue.length == 0) ? null : new String(cacheByteValue, UTF_8); } else { final RocksDB db = rocksDBManager.getDB(extractionNamespace.getNamespace()); @@ -112,17 +102,17 @@ public String apply(@NotNull String val) LOG.error("RocksDB instance is null"); return null; } - byte[] cacheByteValue = db.get(dimension.getBytes()); + byte[] cacheByteValue = db.get(key.getBytes()); if (cacheByteValue == null || cacheByteValue.length == 0) { // No need to call handleMissingLookup if missing dimension is already present in missingLookupCache - if(extractionNamespace.getMissingLookupConfig() != null + if (extractionNamespace.getMissingLookupConfig() != null && !Strings.isNullOrEmpty(extractionNamespace.getMissingLookupConfig().getMissingLookupKafkaTopic()) - && missingLookupCache.getIfPresent(dimension) == null) { + && missingLookupCache.getIfPresent(key) == null) { kafkaManager.handleMissingLookup(extractionNamespaceAsByteArray, extractionNamespace.getMissingLookupConfig().getMissingLookupKafkaTopic(), - dimension); - missingLookupCache.put(dimension, extractionNamespaceAsByteArray); + key); + missingLookupCache.put(key, extractionNamespaceAsByteArray); serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_PUBLISH_MISSING_LOOKUP_SUCESS, 1)); } return null; @@ -153,20 +143,17 @@ private String handleDecode(DecodeConfig decodeConfig, byte[] cacheByteValue, St Descriptors.FieldDescriptor columnIfValueNotMatched = descriptor.findFieldByName(decodeConfig.getColumnIfValueNotMatched()); return Strings.emptyToNull(message.getField(columnIfValueNotMatched).toString()); } - } - else { + } else { Descriptors.FieldDescriptor field = descriptor.findFieldByName(valueColumn); return Strings.emptyToNull(message.getField(field).toString()); } } @Override - public List unapply(final String value) - { - return Lists.newArrayList(Maps.filterKeys(map, new Predicate() - { - @Override public boolean apply(@Nullable String key) - { + public List unapply(final String value) { + return Lists.newArrayList(Maps.filterKeys(map, new Predicate() { + @Override + public boolean apply(@Nullable String key) { return map.get(key).equals(Strings.nullToEmpty(value)); } }).keySet()); @@ -174,28 +161,20 @@ public List unapply(final String value) } @Override - public byte[] getCacheKey() - { + public byte[] getCacheKey() { try { final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - for (Map.Entry entry : map.entrySet()) { - final String key = entry.getKey(); - if (!Strings.isNullOrEmpty(key)) { - outputStream.write(key.getBytes()); - } - outputStream.write((byte)0xFF); - } + outputStream.write(extractionNamespace.toString().getBytes()); + outputStream.write((byte) 0xFF); return outputStream.toByteArray(); - } - catch (IOException ex) { + } catch (IOException ex) { // If ByteArrayOutputStream.write has problems, that is a very bad thing throw Throwables.propagate(ex); } } @Override - public boolean equals(Object o) - { + public boolean equals(Object o) { if (this == o) { return true; } @@ -209,8 +188,7 @@ public boolean equals(Object o) } @Override - public int hashCode() - { + public int hashCode() { return map.hashCode(); } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java index 3b13733a1..6ffa7972c 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java @@ -4,6 +4,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.ImmutableList; @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @@ -16,4 +17,7 @@ public interface ExtractionNamespace { String getLookupName(); + String getTsColumn(); + + boolean isCacheEnabled(); } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/JDBCExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/JDBCExtractionNamespace.java index 51ba26007..cc4820cc3 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/JDBCExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/JDBCExtractionNamespace.java @@ -6,6 +6,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.druid.metadata.MetadataStorageConnectorConfig; import org.joda.time.Period; @@ -16,10 +17,10 @@ import java.sql.Timestamp; import java.util.ArrayList; import java.util.Map; +import java.util.Objects; @JsonTypeName("mahajdbc") -public class JDBCExtractionNamespace implements ExtractionNamespace -{ +public class JDBCExtractionNamespace implements OnlineDatastoreExtractionNamespace { @JsonProperty private final MetadataStorageConnectorConfig connectorConfig; @JsonProperty @@ -29,7 +30,7 @@ public class JDBCExtractionNamespace implements ExtractionNamespace @JsonProperty private final Period pollPeriod; @JsonProperty - private final ArrayList columnList; + private final ImmutableList columnList; @JsonProperty private final String primaryKeyColumn; @JsonProperty @@ -39,32 +40,23 @@ public class JDBCExtractionNamespace implements ExtractionNamespace private boolean firstTimeCaching = true; private Timestamp previousLastUpdateTimestamp; - private final Map columnIndexMap; + private final ImmutableMap columnIndexMap; @JsonCreator public JDBCExtractionNamespace( - @NotNull @JsonProperty(value = "connectorConfig", required = true) - final MetadataStorageConnectorConfig connectorConfig, - @NotNull @JsonProperty(value = "table", required = true) - final String table, - @NotNull @JsonProperty(value = "columnList", required = true) - final ArrayList columnList, - @NotNull @JsonProperty(value = "primaryKeyColumn", required = true) - final String primaryKeyColumn, - @Nullable @JsonProperty(value = "tsColumn", required = false) - final String tsColumn, - @Min(0) @Nullable @JsonProperty(value = "pollPeriod", required = false) - final Period pollPeriod, - @JsonProperty(value = "cacheEnabled", required = false) - final boolean cacheEnabled, - @NotNull @JsonProperty(value = "lookupName", required = true) - final String lookupName - ) - { + @NotNull @JsonProperty(value = "connectorConfig", required = true) final MetadataStorageConnectorConfig connectorConfig, + @NotNull @JsonProperty(value = "table", required = true) final String table, + @NotNull @JsonProperty(value = "columnList", required = true) final ArrayList columnList, + @NotNull @JsonProperty(value = "primaryKeyColumn", required = true) final String primaryKeyColumn, + @Nullable @JsonProperty(value = "tsColumn", required = false) final String tsColumn, + @Min(0) @Nullable @JsonProperty(value = "pollPeriod", required = false) final Period pollPeriod, + @JsonProperty(value = "cacheEnabled", required = false) final boolean cacheEnabled, + @NotNull @JsonProperty(value = "lookupName", required = true) final String lookupName + ) { this.connectorConfig = Preconditions.checkNotNull(connectorConfig, "connectorConfig"); Preconditions.checkNotNull(connectorConfig.getConnectURI(), "connectorConfig.connectURI"); this.table = Preconditions.checkNotNull(table, "table"); - this.columnList = Preconditions.checkNotNull(columnList, "columnList"); + this.columnList = ImmutableList.copyOf(Preconditions.checkNotNull(columnList, "columnList")); this.primaryKeyColumn = Preconditions.checkNotNull(primaryKeyColumn, "primaryKeyColumn"); this.tsColumn = tsColumn; this.pollPeriod = pollPeriod == null ? new Period(0L) : pollPeriod; @@ -72,7 +64,7 @@ public JDBCExtractionNamespace( this.lookupName = lookupName; int index = 0; ImmutableMap.Builder builder = ImmutableMap.builder(); - for(String col : columnList) { + for (String col : columnList) { builder.put(col, index); index += 1; } @@ -80,23 +72,25 @@ public JDBCExtractionNamespace( } public int getColumnIndex(String valueColumn) { - if(columnIndexMap != null && valueColumn != null && columnIndexMap.containsKey(valueColumn)) { + if (columnIndexMap != null && valueColumn != null && columnIndexMap.containsKey(valueColumn)) { return columnIndexMap.get(valueColumn); } return -1; } - public MetadataStorageConnectorConfig getConnectorConfig() - { + public ImmutableMap getColumnIndexMap() { + return columnIndexMap; + } + + public MetadataStorageConnectorConfig getConnectorConfig() { return connectorConfig; } - public String getTable() - { + public String getTable() { return table; } - public ArrayList getColumnList() { + public ImmutableList getColumnList() { return columnList; } @@ -104,8 +98,7 @@ public String getPrimaryKeyColumn() { return primaryKeyColumn; } - public String getTsColumn() - { + public String getTsColumn() { return tsColumn; } @@ -114,8 +107,7 @@ public boolean isCacheEnabled() { } @Override - public long getPollMs() - { + public long getPollMs() { return pollPeriod.toStandardDuration().getMillis(); } @@ -141,62 +133,39 @@ public void setPreviousLastUpdateTimestamp(Timestamp previousLastUpdateTimestamp } @Override - public String toString() - { - return String.format( - "JDBCExtractionNamespace = { connectorConfig = { %s }, table = %s, primaryKeyColumn = %s, columnList = %s, tsColumn = %s, pollPeriod = %s, lookupName = %s}", - connectorConfig.toString(), - table, - primaryKeyColumn, - columnList, - tsColumn, - pollPeriod, - lookupName - ); + public String toString() { + return "JDBCExtractionNamespace{" + + "connectorConfig=" + connectorConfig + + ", table='" + table + '\'' + + ", tsColumn='" + tsColumn + '\'' + + ", pollPeriod=" + pollPeriod + + ", columnList=" + columnList + + ", primaryKeyColumn='" + primaryKeyColumn + '\'' + + ", cacheEnabled=" + cacheEnabled + + ", lookupName='" + lookupName + '\'' + + ", firstTimeCaching=" + firstTimeCaching + + ", previousLastUpdateTimestamp=" + previousLastUpdateTimestamp + + '}'; } @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; JDBCExtractionNamespace that = (JDBCExtractionNamespace) o; - - if (!columnList.equals(that.columnList)) { - return false; - } - if (!table.equals(that.table)) { - return false; - } - if (!primaryKeyColumn.equals(that.primaryKeyColumn)) { - return false; - } - if (tsColumn != null ? !tsColumn.equals(that.tsColumn) : that.tsColumn != null) { - return false; - } - if(!lookupName.equals(that.lookupName)) { - return false; - } - return pollPeriod.equals(that.pollPeriod); - + return isCacheEnabled() == that.isCacheEnabled() && + Objects.equals(getConnectorConfig(), that.getConnectorConfig()) && + Objects.equals(getTable(), that.getTable()) && + Objects.equals(getTsColumn(), that.getTsColumn()) && + Objects.equals(pollPeriod, that.pollPeriod) && + Objects.equals(getColumnList(), that.getColumnList()) && + Objects.equals(getPrimaryKeyColumn(), that.getPrimaryKeyColumn()) && + Objects.equals(getLookupName(), that.getLookupName()); } @Override - public int hashCode() - { - int result = connectorConfig.hashCode(); - result = 31 * result + table.hashCode(); - result = 31 * result + primaryKeyColumn.hashCode(); - result = 31 * result + columnList.hashCode(); - result = 31 * result + (tsColumn != null ? tsColumn.hashCode() : 0); - result = 31 * result + lookupName.hashCode(); - result = 31 * result + pollPeriod.hashCode(); - return result; + public int hashCode() { + return Objects.hash(getConnectorConfig(), getTable(), getTsColumn(), pollPeriod, getColumnList(), getPrimaryKeyColumn(), isCacheEnabled(), getLookupName()); } } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespace.java new file mode 100644 index 000000000..8e2c91848 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespace.java @@ -0,0 +1,195 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.query.lookup.namespace; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.MongoDocumentProcessor; +import org.apache.commons.lang.StringUtils; +import org.joda.time.Period; + +import javax.annotation.Nullable; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; +import java.util.List; +import java.util.Objects; + +@JsonTypeName("mahamongo") +public class MongoExtractionNamespace implements OnlineDatastoreExtractionNamespace { + private static final int DEFAULT_MONGO_CLIENT_RETRY_COUNT = 10; + + private final MongoStorageConnectorConfig connectorConfig; + private final String collectionName; + private final String tsColumn; + private final Period pollPeriod; + private boolean cacheEnabled = true; + private final String lookupName; + private final MongoDocumentProcessor documentProcessor; + private final int mongoClientRetryCount; + private final boolean tsColumnEpochInteger; + + private boolean firstTimeCaching = true; + private long previousLastUpdateTime; + private final ImmutableMap columnIndexMap; + + @JsonCreator + public MongoExtractionNamespace( + @NotNull @JsonProperty(value = "connectorConfig", required = true) final MongoStorageConnectorConfig connectorConfig, + @NotNull @JsonProperty(value = "collectionName", required = true) final String collectionName, + @NotNull @JsonProperty(value = "tsColumn", required = true) final String tsColumn, + @NotNull @JsonProperty(value = "tsColumnEpochInteger", required = true) final boolean tsColumnEpochInteger, + @Min(0) @Nullable @JsonProperty(value = "pollPeriod", required = false) final Period pollPeriod, + @JsonProperty(value = "cacheEnabled", required = false) final boolean cacheEnabled, + @NotNull @JsonProperty(value = "lookupName", required = true) final String lookupName, + @NotNull @JsonProperty(value = "documentProcessor", required = true) final MongoDocumentProcessor documentProcessor, + @Nullable @JsonProperty(value = "mongoClientRetryCount", required = false) final Integer mongoClientRetryCount + ) { + this.connectorConfig = Preconditions.checkNotNull(connectorConfig, "connectorConfig"); + Preconditions.checkNotNull(connectorConfig.getConnectURI(), "connectorConfig.connectURI"); + this.documentProcessor = documentProcessor; + Preconditions.checkArgument( + StringUtils.isNotBlank(collectionName), "collectionName must be provided"); + Preconditions.checkArgument( + StringUtils.isNotBlank(tsColumn), "tsColumn must be provided"); + if (mongoClientRetryCount == null || mongoClientRetryCount <= 0) { + this.mongoClientRetryCount = DEFAULT_MONGO_CLIENT_RETRY_COUNT; + } else { + this.mongoClientRetryCount = mongoClientRetryCount; + } + this.collectionName = collectionName; + this.tsColumn = tsColumn; + this.tsColumnEpochInteger = tsColumnEpochInteger; + this.pollPeriod = pollPeriod == null ? new Period(0L) : pollPeriod; + this.cacheEnabled = cacheEnabled; + this.lookupName = lookupName; + int index = 0; + ImmutableMap.Builder builder = ImmutableMap.builder(); + List columnList = documentProcessor.getColumnList(); + for (String col : columnList) { + builder.put(col, index); + index += 1; + } + this.columnIndexMap = builder.build(); + } + + @JsonIgnore + public int getColumnIndex(String valueColumn) { + if (columnIndexMap != null && valueColumn != null && columnIndexMap.containsKey(valueColumn)) { + return columnIndexMap.get(valueColumn); + } + return -1; + } + + @JsonIgnore + public ImmutableMap getColumnIndexMap() { + return columnIndexMap; + } + + public MongoStorageConnectorConfig getConnectorConfig() { + return connectorConfig; + } + + public String getCollectionName() { + return collectionName; + } + + public List getColumnList() { + return documentProcessor.getColumnList(); + } + + public String getPrimaryKeyColumn() { + return documentProcessor.getPrimaryKeyColumn(); + } + + public String getTsColumn() { + return tsColumn; + } + + public boolean isTsColumnEpochInteger() { + return tsColumnEpochInteger; + } + + public boolean isCacheEnabled() { + return cacheEnabled; + } + + @Override + public long getPollMs() { + return pollPeriod.toStandardDuration().getMillis(); + } + + @Override + public String getLookupName() { + return lookupName; + } + + public MongoDocumentProcessor getDocumentProcessor() { + return documentProcessor; + } + + public int getMongoClientRetryCount() { + return mongoClientRetryCount; + } + + public boolean isFirstTimeCaching() { + return firstTimeCaching; + } + + private void setFirstTimeCaching(boolean value) { + this.firstTimeCaching = value; + } + + @JsonIgnore + public long getPreviousLastUpdateTime() { + return previousLastUpdateTime; + } + + @JsonIgnore + public void setPreviousLastUpdateTime(long previousLastUpdateTime) { + this.previousLastUpdateTime = previousLastUpdateTime; + setFirstTimeCaching(false); + } + + @Override + public String toString() { + return "MongoExtractionNamespace{" + + "connectorConfig=" + connectorConfig + + ", collectionName='" + collectionName + '\'' + + ", tsColumn='" + tsColumn + '\'' + + ", pollPeriod=" + pollPeriod + + ", cacheEnabled=" + cacheEnabled + + ", lookupName='" + lookupName + '\'' + + ", documentProcessor=" + documentProcessor + + ", mongoClientRetryCount=" + mongoClientRetryCount + + ", tsColumnEpochInteger=" + tsColumnEpochInteger + + ", firstTimeCaching=" + firstTimeCaching + + ", previousLastUpdateTime=" + previousLastUpdateTime + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + MongoExtractionNamespace that = (MongoExtractionNamespace) o; + return isCacheEnabled() == that.isCacheEnabled() && + getMongoClientRetryCount() == that.getMongoClientRetryCount() && + isTsColumnEpochInteger() == that.isTsColumnEpochInteger() && + Objects.equals(getConnectorConfig(), that.getConnectorConfig()) && + Objects.equals(getCollectionName(), that.getCollectionName()) && + Objects.equals(getTsColumn(), that.getTsColumn()) && + Objects.equals(pollPeriod, that.pollPeriod) && + Objects.equals(getLookupName(), that.getLookupName()) && + Objects.equals(getDocumentProcessor(), that.getDocumentProcessor()); + } + + @Override + public int hashCode() { + return Objects.hash(getConnectorConfig(), getCollectionName(), getTsColumn(), pollPeriod, isCacheEnabled(), getLookupName(), getDocumentProcessor(), getMongoClientRetryCount(), isTsColumnEpochInteger()); + } +} + diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfig.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfig.java new file mode 100644 index 000000000..2ef2f8d42 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfig.java @@ -0,0 +1,202 @@ +package com.yahoo.maha.maha_druid_lookups.query.lookup.namespace; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.Lists; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientOptions; +import com.mongodb.MongoCredential; +import com.mongodb.ServerAddress; +import io.druid.metadata.PasswordProvider; + +import java.lang.reflect.Method; +import java.util.*; +import java.util.stream.Collectors; + +public class MongoStorageConnectorConfig { + + /** + * host1:port1,host2:port2,host3:port3 example usage: MongoClient mongoClient = new + * MongoClient(Arrays.asList(new ServerAddress("host1", port1), new ServerAddress("host2", port2), + * new ServerAddress("host3", port3))); + */ + @JsonProperty + private String hosts = "localhost"; + + /** + * Name of db in mongo, e.g. mydb example usage: DB db = mongoClient.getDB( "mydb" ); + */ + @JsonProperty + private String dbName; + + /** + * user name used for authentication example usage: MongoCredential credential = + * MongoCredential.createCredential(userName, database, password); MongoClient mongoClient = new + * MongoClient(new ServerAddress(), Arrays.asList(credential)); + */ + @JsonProperty + private String user = null; + + /** + * password provided by PasswordProvider example usage: MongoCredential credential = + * MongoCredential.createCredential(userName, database, password); MongoClient mongoClient = new + * MongoClient(new ServerAddress(), Arrays.asList(credential)); + */ + @JsonProperty("password") + private PasswordProvider passwordProvider; + + /** + * Map of client options to set example usage: MongoClient mongoClient = new MongoClient(new + * ServerAddress(), Arrays.asList(credential), mongoClientOptions); + */ + @JsonProperty("clientOptions") + private Properties clientProperties; + + public String getHosts() { + return hosts; + } + + public String getDbName() { + return dbName; + } + + public String getUser() { + return user; + } + + public String getPassword() { + return passwordProvider == null ? null : passwordProvider.getPassword(); + } + + public String getConnectURI() { + Preconditions.checkState(!Strings.isNullOrEmpty(this.getHosts()), "Hosts is null or empty"); + Preconditions.checkState(!Strings.isNullOrEmpty(this.getDbName()), "Db name is null or empty"); + return "mongodb://" + (Strings.isNullOrEmpty(this.getPassword()) ? this.getHosts() + : this.getUser() + ":" + this.getPassword() + "@" + this.getHosts()) + "/" + this + .getDbName(); + } + + public Properties getClientProperties() { + return clientProperties; + } + + static final String[] INT_PROPERTIES = new String[]{ + "connectionsPerHost" + , "connectTimeout" + , "heartbeatConnectTimeout" + , "heartbeatFrequency" + , "heartbeatSocketTimeout" + , "localThreshold" + , "maxConnectionIdleTime" + , "maxConnectionLifeTime" + , "maxWaitTime" + , "minConnectionsPerHost" + , "minHeartbeatFrequency" + , "serverSelectionTimeout" + , "socketTimeout" + , "threadsAllowedToBlockForConnectionMultiplier" + }; + + static final String[] BOOL_PROPERTIES = new String[]{ + "alwaysUseMBeans" + , "cursorFinalizerEnabled" + , "socketKeepAlive" + , "sslEnabled" + , "sslInvalidHostNameAllowed" + }; + + @VisibleForTesting + MongoClientOptions getMongoClientOptions() { + MongoClientOptions.Builder builder = MongoClientOptions.builder(); + + if (getClientProperties() != null) { + Class clazz = MongoClientOptions.Builder.class; + Properties props = getClientProperties(); + for (String p : INT_PROPERTIES) { + String value = props.getProperty(p); + if (value != null) { + try { + int i = Integer.parseInt(value); + Method m = clazz.getMethod(p, int.class); + m.invoke(builder, i); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to parse int for property : %s=%s", p, value), e); + } + } + } + for (String p : BOOL_PROPERTIES) { + String value = props.getProperty(p); + if (value != null) { + try { + boolean b = Boolean.parseBoolean(value); + Method m = clazz.getMethod(p, boolean.class); + m.invoke(builder, b); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to parse boolean for property : %s=%s", p, value), e); + } + } + } + } + + return builder.build(); + } + + @Override + public String toString() { + return "MongoStorageConnectorConfig{" + + "hosts='" + hosts + '\'' + + ", dbName='" + dbName + '\'' + + ", user='" + user + '\'' + + ", passwordProvider=" + passwordProvider + + ", clientProperties=" + clientProperties + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MongoStorageConnectorConfig that = (MongoStorageConnectorConfig) o; + return Objects.equals(getHosts(), that.getHosts()) && + Objects.equals(getDbName(), that.getDbName()) && + Objects.equals(getUser(), that.getUser()) && + Objects.equals(passwordProvider, that.passwordProvider) && + Objects.equals(getClientProperties(), that.getClientProperties()); + } + + @Override + public int hashCode() { + return Objects + .hash(getHosts(), getDbName(), getUser(), passwordProvider, getClientProperties()); + } + + public MongoClient getMongoClient() { + MongoStorageConnectorConfig config = this; + List serverAddressList = Arrays + .stream(config.getHosts().split(",")).map(s -> { + String[] hostPort = s.split(":"); + if (hostPort.length > 1) { + return new ServerAddress(hostPort[0], Integer.parseInt(hostPort[1])); + } else { + return new ServerAddress(hostPort[0]); + } + }).collect(Collectors.toList()); + + List mongoCredentialList = Collections.emptyList(); + if (getUser() != null && getPassword() != null) { + mongoCredentialList = Lists.newArrayList(MongoCredential + .createCredential(config.getUser(), config.getDbName(), config.getPassword().toCharArray())); + + } + MongoClientOptions options = config.getMongoClientOptions(); + return new MongoClient(serverAddressList, mongoCredentialList, options); + } +} \ No newline at end of file diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/OnlineDatastoreExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/OnlineDatastoreExtractionNamespace.java new file mode 100644 index 000000000..a2a1d5480 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/OnlineDatastoreExtractionNamespace.java @@ -0,0 +1,20 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.query.lookup.namespace; + +import com.google.common.collect.ImmutableMap; + +public interface OnlineDatastoreExtractionNamespace extends ExtractionNamespace { + + long getPollMs(); + + String getLookupName(); + + String getTsColumn(); + + boolean isCacheEnabled(); + + String getPrimaryKeyColumn(); + + ImmutableMap getColumnIndexMap(); +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java index 1ff7c4ef6..d9de81dd5 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespace.java @@ -41,25 +41,17 @@ public class RocksDBExtractionNamespace implements ExtractionNamespace { @JsonCreator public RocksDBExtractionNamespace(@NotNull @JsonProperty(value = "namespace", required = true) - String namespace, - @NotNull @JsonProperty(value = "rocksDbInstanceHDFSPath", required = true) - final String rocksDbInstanceHDFSPath, - @NotNull @JsonProperty(value = "lookupAuditingHDFSPath", required = true) - final String lookupAuditingHDFSPath, - @Min(0) @JsonProperty(value = "pollPeriod", required = true) - Period pollPeriod, - @NotNull @JsonProperty(value = "kafkaTopic", required = false) - final String kafkaTopic, - @JsonProperty(value = "cacheEnabled", required = false) - final boolean cacheEnabled, - @JsonProperty(value = "lookupAuditingEnabled", required = false) - final boolean lookupAuditingEnabled, - @NotNull @JsonProperty(value = "lookupName", required = true) - final String lookupName, - @Nullable @JsonProperty(value = "tsColumn", required = false) - final String tsColumn, - @NotNull @JsonProperty(value = "missingLookupConfig", required = false) - final MissingLookupConfig missingLookupConfig) { + String namespace, + @NotNull @JsonProperty(value = "rocksDbInstanceHDFSPath", required = true) final String rocksDbInstanceHDFSPath, + @NotNull @JsonProperty(value = "lookupAuditingHDFSPath", required = true) final String lookupAuditingHDFSPath, + @Min(0) @JsonProperty(value = "pollPeriod", required = true) + Period pollPeriod, + @NotNull @JsonProperty(value = "kafkaTopic", required = false) final String kafkaTopic, + @JsonProperty(value = "cacheEnabled", required = false) final boolean cacheEnabled, + @JsonProperty(value = "lookupAuditingEnabled", required = false) final boolean lookupAuditingEnabled, + @NotNull @JsonProperty(value = "lookupName", required = true) final String lookupName, + @Nullable @JsonProperty(value = "tsColumn", required = false) final String tsColumn, + @NotNull @JsonProperty(value = "missingLookupConfig", required = false) final MissingLookupConfig missingLookupConfig) { this.rocksDbInstanceHDFSPath = Preconditions.checkNotNull(rocksDbInstanceHDFSPath, "rocksDbInstanceHDFSPath"); this.lookupAuditingHDFSPath = Preconditions.checkNotNull(lookupAuditingHDFSPath, "lookupAuditingHDFSPath"); this.namespace = Preconditions.checkNotNull(namespace, "namespace"); @@ -120,21 +112,25 @@ public void setLastUpdatedTime(Long lastUpdatedTime) { this.lastUpdatedTime = lastUpdatedTime; } - public String getTsColumn() - { + public String getTsColumn() { return tsColumn; } @Override public String toString() { - return String.format( - "RocksDBExtractionNamespace = { namespace = %s, rocksDbInstanceHDFSPath = { %s }, pollPeriod = %s, kafkaTopic = %s, missingLookupConfig = %s }", - namespace, - rocksDbInstanceHDFSPath, - pollPeriod, - kafkaTopic, - missingLookupConfig - ); + return "RocksDBExtractionNamespace{" + + "rocksDbInstanceHDFSPath='" + rocksDbInstanceHDFSPath + '\'' + + ", lookupAuditingHDFSPath='" + lookupAuditingHDFSPath + '\'' + + ", namespace='" + namespace + '\'' + + ", pollPeriod=" + pollPeriod + + ", kafkaTopic='" + kafkaTopic + '\'' + + ", cacheEnabled=" + cacheEnabled + + ", lookupAuditingEnabled=" + lookupAuditingEnabled + + ", lookupName='" + lookupName + '\'' + + ", tsColumn='" + tsColumn + '\'' + + ", missingLookupConfig=" + missingLookupConfig + + ", lastUpdatedTime=" + lastUpdatedTime + + '}'; } @Override @@ -151,8 +147,7 @@ public boolean equals(Object o) { Objects.equals(kafkaTopic, that.kafkaTopic) && Objects.equals(lookupName, that.lookupName) && Objects.equals(tsColumn, that.tsColumn) && - Objects.equals(missingLookupConfig, that.missingLookupConfig) && - Objects.equals(lastUpdatedTime, that.lastUpdatedTime); + Objects.equals(missingLookupConfig, that.missingLookupConfig); } @Override @@ -167,7 +162,6 @@ public int hashCode() { lookupAuditingEnabled, lookupName, tsColumn, - missingLookupConfig, - lastUpdatedTime); + missingLookupConfig); } } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java index b36a7135f..fab7765f8 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/JDBCExtractionNamespaceCacheFactory.java @@ -2,11 +2,9 @@ // Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; -import com.google.common.base.Strings; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; -import com.metamx.emitter.service.ServiceMetricEvent; import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; @@ -16,7 +14,6 @@ import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.util.TimestampMapper; -import java.io.UnsupportedEncodingException; import java.sql.Timestamp; import java.util.List; import java.util.Map; @@ -29,8 +26,7 @@ * */ public class JDBCExtractionNamespaceCacheFactory - implements ExtractionNamespaceCacheFactory> -{ + implements ExtractionNamespaceCacheFactory> { private static final Logger LOG = new Logger(JDBCExtractionNamespaceCacheFactory.class); private static final String COMMA_SEPARATOR = ","; private static final String FIRST_TIME_CACHING_WHERE_CLAUSE = " WHERE LAST_UPDATED <= :lastUpdatedTimeStamp"; @@ -48,9 +44,8 @@ public Callable getCachePopulator( final JDBCExtractionNamespace extractionNamespace, final String lastVersion, final Map> cache - ) - { - final long lastCheck = lastVersion == null ? Long.MIN_VALUE/2 : Long.parseLong(lastVersion); + ) { + final long lastCheck = lastVersion == null ? Long.MIN_VALUE / 2 : Long.parseLong(lastVersion); if (!extractionNamespace.isCacheEnabled()) { return new Callable() { @Override @@ -61,30 +56,24 @@ public String call() throws Exception { } final Timestamp lastDBUpdate = lastUpdates(id, extractionNamespace); if (lastDBUpdate != null && lastDBUpdate.getTime() <= lastCheck) { - return new Callable() - { + return new Callable() { @Override - public String call() throws Exception - { + public String call() throws Exception { extractionNamespace.setPreviousLastUpdateTimestamp(lastDBUpdate); return lastVersion; } }; } - return new Callable() - { + return new Callable() { @Override - public String call() - { + public String call() { final DBI dbi = ensureDBI(id, extractionNamespace); LOG.debug("Updating [%s]", id); dbi.withHandle( - new HandleCallback() - { + new HandleCallback() { @Override - public Void withHandle(Handle handle) throws Exception - { + public Void withHandle(Handle handle) throws Exception { String query = String.format("SELECT %s FROM %s", String.join(COMMA_SEPARATOR, extractionNamespace.getColumnList()), extractionNamespace.getTable() @@ -120,8 +109,7 @@ public Void withHandle(Handle handle) throws Exception }; } - private DBI ensureDBI(String id, JDBCExtractionNamespace namespace) - { + private DBI ensureDBI(String id, JDBCExtractionNamespace namespace) { final String key = id; DBI dbi = null; if (dbiCache.containsKey(key)) { @@ -139,8 +127,7 @@ private DBI ensureDBI(String id, JDBCExtractionNamespace namespace) return dbi; } - private Timestamp lastUpdates(String id, JDBCExtractionNamespace namespace) - { + private Timestamp lastUpdates(String id, JDBCExtractionNamespace namespace) { final DBI dbi = ensureDBI(id, namespace); final String table = namespace.getTable(); final String tsColumn = namespace.getTsColumn(); @@ -148,8 +135,7 @@ private Timestamp lastUpdates(String id, JDBCExtractionNamespace namespace) return null; } final Timestamp lastUpdatedTimeStamp = dbi.withHandle( - new HandleCallback() - { + new HandleCallback() { @Override public Timestamp withHandle(Handle handle) throws Exception { @@ -183,16 +169,16 @@ public byte[] getCacheValue(final JDBCExtractionNamespace extractionNamespace, f return value; } List cacheValue = cache.get(key); - if(cacheValue == null) { + if (cacheValue == null) { return new byte[0]; } - if(decodeConfigOptional.isPresent()) { + if (decodeConfigOptional.isPresent()) { return handleDecode(extractionNamespace, cacheValue, decodeConfigOptional.get()); } int index = extractionNamespace.getColumnIndex(valueColumn); - if(index == -1) { + if (index == -1) { LOG.error("invalid valueColumn [%s]", valueColumn); return new byte[0]; } @@ -203,13 +189,13 @@ public byte[] getCacheValue(final JDBCExtractionNamespace extractionNamespace, f private byte[] handleDecode(JDBCExtractionNamespace extractionNamespace, List cacheValue, DecodeConfig decodeConfig) { final int columnToCheckIndex = extractionNamespace.getColumnIndex(decodeConfig.getColumnToCheck()); - if (columnToCheckIndex < 0 || columnToCheckIndex >= cacheValue.size() ) { + if (columnToCheckIndex < 0 || columnToCheckIndex >= cacheValue.size()) { return new byte[0]; } final String valueFromColumnToCheck = cacheValue.get(columnToCheckIndex); - if(valueFromColumnToCheck != null && valueFromColumnToCheck.equals(decodeConfig.getValueToCheck())) { + if (valueFromColumnToCheck != null && valueFromColumnToCheck.equals(decodeConfig.getValueToCheck())) { final int columnIfValueMatchedIndex = extractionNamespace.getColumnIndex(decodeConfig.getColumnIfValueMatched()); if (columnIfValueMatchedIndex < 0) { return new byte[0]; diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java index 60bef732c..276002c1b 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/KafkaManager.java @@ -14,7 +14,7 @@ import com.metamx.common.logger.Logger; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; import io.druid.guice.ManageLifecycle; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -54,14 +54,14 @@ public class KafkaManager { .setPriority(Thread.MIN_PRIORITY) .build()); - private final Provider namespaceExtractionCacheManager; + private final Provider namespaceExtractionCacheManager; private final ProtobufSchemaFactory protobufSchemaFactory; private KafkaProducer kafkaProducer; @Inject - public KafkaManager(Provider namespaceExtractionCacheManager, + public KafkaManager(Provider namespaceExtractionCacheManager, final MahaNamespaceExtractionConfig mahaNamespaceExtractionConfig, ProtobufSchemaFactory protobufSchemaFactory) { this.kafkaProperties.putAll(mahaNamespaceExtractionConfig.getKafkaProperties()); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java index 116b3511f..76f7bb1a5 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java @@ -2,23 +2,26 @@ // Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; +import com.fasterxml.jackson.annotation.JsonProperty; + import java.util.Properties; -import org.codehaus.jackson.annotate.JsonProperty; +import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; public class MahaNamespaceExtractionConfig { - @JsonProperty("kafka") + @Nullable @JsonProperty(value = "kafka") private Properties kafkaProperties; - @JsonProperty("service") + @Nullable @JsonProperty(value = "service") private Properties lookupServiceProperties; - @JsonProperty("rocksdb") + @Nullable @JsonProperty(value = "rocksdb") private Properties rocksDBProperties; - @JsonProperty("schemaFactory") + @NotNull @JsonProperty(value = "schemaFactory") private String protobufSchemaFactoryClass; - @JsonProperty("authHeaderFactory") + @NotNull @JsonProperty(value = "authHeaderFactory") private String authHeaderFactoryClass; public Properties getKafkaProperties() { diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java index fd0796951..c5f0ea2c9 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java @@ -15,8 +15,8 @@ import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaNamespaceExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; @@ -32,7 +32,8 @@ */ public class MahaNamespaceExtractionModule implements DruidModule { - public static final String TYPE_PREFIX = "druid.lookup.namespace.cache.type"; + public static final String PREFIX = "druid.lookup.maha.namespace"; + public static final String TYPE_PREFIX = "druid.lookup.maha.namespace.cache.type"; @Override public List getJacksonModules() @@ -62,16 +63,16 @@ public static MapBinder, ExtractionNamespac @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.lookup.maha.namespace", MahaNamespaceExtractionConfig.class); + JsonConfigProvider.bind(binder, PREFIX, MahaNamespaceExtractionConfig.class); PolyBind - .createChoiceWithDefault(binder, TYPE_PREFIX, Key.get(MahaExtractionCacheManager.class), "onHeap") + .createChoiceWithDefault(binder, TYPE_PREFIX, Key.get(MahaNamespaceExtractionCacheManager.class), "onHeap") .in(LazySingleton.class); PolyBind - .optionBinder(binder, Key.get(MahaExtractionCacheManager.class)) + .optionBinder(binder, Key.get(MahaNamespaceExtractionCacheManager.class)) .addBinding("onHeap") - .to(OnHeapMahaExtractionCacheManager.class) + .to(OnHeapMahaNamespaceExtractionCacheManager.class) .in(LazySingleton.class); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java index 7de1af19e..c4dea1559 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java @@ -11,7 +11,7 @@ import com.metamx.emitter.service.ServiceMetricEvent; import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; import io.druid.server.security.Access; import io.druid.server.security.AuthConfig; @@ -33,13 +33,13 @@ public class MahaNamespacesCacheResource { private static final Logger log = new Logger(MahaNamespacesCacheResource.class); private static final ObjectMapper objectMapper = new ObjectMapper(); - private final MahaExtractionCacheManager mahaExtractionCacheManager; + private final MahaNamespaceExtractionCacheManager mahaNamespaceExtractionCacheManager; private final ServiceEmitter serviceEmitter; @Inject - public MahaNamespacesCacheResource(final MahaExtractionCacheManager mahaExtractionCacheManager, + public MahaNamespacesCacheResource(final MahaNamespaceExtractionCacheManager mahaNamespaceExtractionCacheManager, final ServiceEmitter serviceEmitter){ - this.mahaExtractionCacheManager = mahaExtractionCacheManager; + this.mahaNamespaceExtractionCacheManager = mahaNamespaceExtractionCacheManager; this.serviceEmitter = serviceEmitter; } @@ -48,10 +48,10 @@ public MahaNamespacesCacheResource(final MahaExtractionCacheManager mahaExtracti public Response getNamespaces(@Context final HttpServletRequest request){ try{ request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, Access.OK.isAllowed()); - Collection namespaces = mahaExtractionCacheManager.getKnownIDs(); + Collection namespaces = mahaNamespaceExtractionCacheManager.getKnownIDs(); Map response = new HashMap(); for(String namespace: namespaces) { - response.put(namespace, mahaExtractionCacheManager.getCacheMap(namespace).size()); + response.put(namespace, mahaNamespaceExtractionCacheManager.getCacheMap(namespace).size()); } return Response.ok().entity(response).build(); }catch (Exception ex){ @@ -72,7 +72,7 @@ public Response getCacheValue(@PathParam("namespace") String namespace, try { request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, Access.OK.isAllowed()); byte[] response; - Optional extractionNamespace = mahaExtractionCacheManager.getExtractionNamespace(namespace); + Optional extractionNamespace = mahaNamespaceExtractionCacheManager.getExtractionNamespace(namespace); if(!extractionNamespace.isPresent()) { return Response.ok().entity(new byte[0]).build(); @@ -92,19 +92,19 @@ public Response getCacheValue(@PathParam("namespace") String namespace, decodeConfigOptional = Optional.of(decodeConfig); } - response = mahaExtractionCacheManager + response = mahaNamespaceExtractionCacheManager .getExtractionNamespaceFunctionFactory(Class.forName(extractionNamespaceClass)) .getCacheValue(extractionNamespace.get(), - mahaExtractionCacheManager.getCacheMap(namespace), key, valueColumn, decodeConfigOptional); + mahaNamespaceExtractionCacheManager.getCacheMap(namespace), key, valueColumn, decodeConfigOptional); if (debug && response != null) { log.info("Cache value is : [%s]", new String(response)); } } else { log.warn("Key is not passed hence returning the size of the cache"); - response = mahaExtractionCacheManager + response = mahaNamespaceExtractionCacheManager .getExtractionNamespaceFunctionFactory(Class.forName(extractionNamespaceClass)) .getCacheSize(extractionNamespace.get(), - mahaExtractionCacheManager.getCacheMap(namespace)).getBytes(); + mahaNamespaceExtractionCacheManager.getCacheMap(namespace)).getBytes(); } serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_GET_CACHE_VALUE_SUCESS, 1)); return Response.ok().entity(response).build(); @@ -125,10 +125,10 @@ public Response getLastUpdatedTime(@PathParam("namespace") String namespace, request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, Access.OK.isAllowed()); log.info("Fetching lastUpdatedTime namespace [%s]", namespace); - Optional extractionNamespace = mahaExtractionCacheManager.getExtractionNamespace(namespace); + Optional extractionNamespace = mahaNamespaceExtractionCacheManager.getExtractionNamespace(namespace); Long lastUpdatedTime = -1L; if(extractionNamespace.isPresent()) { - lastUpdatedTime = mahaExtractionCacheManager + lastUpdatedTime = mahaNamespaceExtractionCacheManager .getExtractionNamespaceFunctionFactory(Class.forName(extractionNamespaceClass)) .getLastUpdatedTime(extractionNamespace.get()); } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java new file mode 100644 index 000000000..93c78be44 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java @@ -0,0 +1,306 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; + +import com.google.inject.Inject; +import com.metamx.common.logger.Logger; +import com.metamx.emitter.service.ServiceEmitter; +import com.mongodb.MongoClient; +import com.mongodb.client.FindIterable; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.Projections; +import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoStorageConnectorConfig; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.LookupBuilder; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.MongoDocumentProcessor; + +import java.util.*; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import org.bson.Document; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.testng.collections.Lists; + +/** + * + */ +public class MongoExtractionNamespaceCacheFactory + implements ExtractionNamespaceCacheFactory> { + private static final DateTimeFormatter ISODATE_FORMATTER = + DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSS"); + private static final int[] BACKOFF_MILLIS = new int[]{100, 200, 400, 800, 1600, 3200, 6400, 12800}; + private static final Logger LOG = new Logger(MongoExtractionNamespaceCacheFactory.class); + private static final String COMMA_SEPARATOR = ","; + private static final String ID_FIELD = "_id"; + //private static final String FIRST_TIME_CACHING_WHERE_CLAUSE = " WHERE LAST_UPDATED <= :lastUpdatedTimeStamp"; + //private static final String SUBSEQUENT_CACHING_WHERE_CLAUSE = " WHERE LAST_UPDATED > :lastUpdatedTimeStamp"; + //private static final int FETCH_SIZE = 10000; + //private final ConcurrentMap dbiCache = new ConcurrentHashMap<>(); + private final ConcurrentMap mongoClientCache = new ConcurrentHashMap<>(); + @Inject + LookupService lookupService; + @Inject + ServiceEmitter emitter; + + @Override + public Callable getCachePopulator( + final String id, + final MongoExtractionNamespace extractionNamespace, + final String lastVersion, + final Map> cache + ) { + final long lastCheck = lastVersion == null ? Long.MIN_VALUE / 2 : Long.parseLong(lastVersion); + if (!extractionNamespace.isCacheEnabled()) { + return new Callable() { + @Override + public String call() throws Exception { + return String.valueOf(lastCheck); + } + }; + } + + return new Callable() { + @Override + public String call() { + if (LOG.isDebugEnabled()) { + LOG.debug("Updating [%s]", id); + } + MongoStorageConnectorConfig config = extractionNamespace.getConnectorConfig(); + int numAttempts = 0; + MongoDocumentProcessor processor = extractionNamespace.getDocumentProcessor(); + Set neededFields = new TreeSet<>(); + neededFields.add(ID_FIELD); + neededFields.add(extractionNamespace.getTsColumn()); + neededFields.add(processor.getPrimaryKeyColumn()); + neededFields.addAll(processor.getColumnList()); + + Date currentDate = new Date(); + long maxTime = -1; + LookupBuilder lookupBuilder = new LookupBuilder(cache); + FindIterable documents; + + MongoDatabase database = null; + MongoCollection collection = null; + while (numAttempts < extractionNamespace.getMongoClientRetryCount()) { + try { + final MongoClient mongoClient = ensureMongoClient(id, extractionNamespace); + if (LOG.isDebugEnabled()) { + LOG.debug("Attempting to get database with config : %s" + , config); + } + database = mongoClient.getDatabase(config.getDbName()); + LOG.info("Successfully got database with hosts=%s database=%s" + , config.getHosts(), + config.getDbName()); + collection = database.getCollection(extractionNamespace.getCollectionName()); + LOG.info("Successfully got collection : %s" + , extractionNamespace.getCollectionName()); + + break; + } catch (Exception e) { + LOG.error(e, "Failed to create mongo client, numAttempt=%s hosts=%s database=%s" + , numAttempts + , extractionNamespace.getConnectorConfig().getHosts() + , extractionNamespace.getConnectorConfig().getDbName() + ); + backOffSleep(numAttempts); + } + numAttempts++; + } + + if (database == null || collection == null) { + throw new RuntimeException(String.format("Failed to get database or collection from mongo client: hosts=%s database=%s collection=%s", + config.getHosts(), config.getDbName(), extractionNamespace.getCollectionName())); + } + + if (extractionNamespace.isFirstTimeCaching()) { + documents = collection.find(); + } else { + if (extractionNamespace.isTsColumnEpochInteger()) { + documents = collection.find(Filters.gte(extractionNamespace.getTsColumn(), extractionNamespace.getPreviousLastUpdateTime())); + } else { + Date date = new Date(extractionNamespace.getPreviousLastUpdateTime()); + documents = collection.find(Filters.gte(extractionNamespace.getTsColumn(), date)); + } + } + documents = documents.projection(Projections.include(Lists.newArrayList(neededFields))); + long docTime = -1; + for (Document d : documents) { + try { + if (extractionNamespace.isTsColumnEpochInteger()) { + docTime = d.getInteger(extractionNamespace.getTsColumn()); + } else { + docTime = d.getDate(extractionNamespace.getTsColumn()).getTime(); + } + try { + processor.process(d, lookupBuilder); + if (maxTime < docTime) { + maxTime = docTime; + } + } catch (Exception e) { + LOG.error(e, "collectionName=%s tsColumn=%s failed to process document document=%s" + , extractionNamespace.getCollectionName() + , extractionNamespace.getTsColumn() + , d.toJson() + ); + } + + } catch (Exception e) { + LOG.error(e, "collectionName=%s tsColumn=%s failed to process document document=%s" + , extractionNamespace.getCollectionName() + , extractionNamespace.getTsColumn() + , d.toJson() + ); + } + + } + + if (extractionNamespace.isTsColumnEpochInteger()) { + if (maxTime < 0 || currentDate.before(new Date(maxTime * 1000))) { + extractionNamespace.setPreviousLastUpdateTime(currentDate.getTime() / 1000); + } else { + extractionNamespace.setPreviousLastUpdateTime(maxTime); + } + } else { + if (maxTime < 0 || currentDate.before(new Date(maxTime))) { + extractionNamespace.setPreviousLastUpdateTime(currentDate.getTime()); + } else { + extractionNamespace.setPreviousLastUpdateTime(maxTime); + } + + } + + LOG.info("Finished loading %d values for extractionNamespace[%s]", cache.size(), id); + return String.format("%d", extractionNamespace.getPreviousLastUpdateTime()); + } + }; + } + + private MongoClient ensureMongoClient(String id, MongoExtractionNamespace namespace) { + final String key = id; + MongoClient mongoClient = null; + if (mongoClientCache.containsKey(key)) { + mongoClient = mongoClientCache.get(key); + } + if (mongoClient == null) { + int numAttempts = 0; + while (numAttempts < namespace.getMongoClientRetryCount()) { + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Attempting to create mongo client with config : %s" + , namespace.getConnectorConfig()); + } + final MongoClient newClient = namespace.getConnectorConfig().getMongoClient(); + LOG.info("Successfully created mongo client with hosts=%s database=%s" + , namespace.getConnectorConfig().getHosts(), + namespace.getConnectorConfig().getDbName()); + mongoClientCache.putIfAbsent(key, newClient); + mongoClient = mongoClientCache.get(key); + if (newClient != mongoClient) { + newClient.close(); + } + break; + } catch (Exception e) { + LOG.error(e, "Failed to create mongo client, numAttempt=%s hosts=%s database=%s" + , numAttempts + , namespace.getConnectorConfig().getHosts() + , namespace.getConnectorConfig().getDbName() + ); + backOffSleep(numAttempts); + } + numAttempts++; + } + } + return mongoClient; + } + + private void backOffSleep(int numAttempts) { + try { + int sleepMillis = BACKOFF_MILLIS[numAttempts % BACKOFF_MILLIS.length]; + Thread.sleep(sleepMillis); + } catch (Exception e) { + LOG.error(e, "Error while sleeping"); + } + } + + @Override + public void updateCache(final MongoExtractionNamespace extractionNamespace, final Map> cache, + final String key, final byte[] value) { + //No-Op + } + + @Override + public byte[] getCacheValue(final MongoExtractionNamespace extractionNamespace, final Map> cache, final String key, final String valueColumn, final Optional decodeConfigOptional) { + if (!extractionNamespace.isCacheEnabled()) { + byte[] value = lookupService.lookup(new LookupService.LookupData(extractionNamespace, key, valueColumn, decodeConfigOptional)); + value = (value == null) ? new byte[0] : value; + LOG.info("Cache value [%s]", new String(value)); + return value; + } + List cacheValue = cache.get(key); + if (cacheValue == null) { + return new byte[0]; + } + + if (decodeConfigOptional.isPresent()) { + return handleDecode(extractionNamespace, cacheValue, decodeConfigOptional.get()); + } + + int index = extractionNamespace.getColumnIndex(valueColumn); + if (index == -1) { + LOG.error("invalid valueColumn [%s]", valueColumn); + return new byte[0]; + } + String value = cacheValue.get(index); + return (value == null) ? new byte[0] : value.getBytes(); + } + + private byte[] handleDecode(MongoExtractionNamespace extractionNamespace, List cacheValue, DecodeConfig decodeConfig) { + + final int columnToCheckIndex = extractionNamespace.getColumnIndex(decodeConfig.getColumnToCheck()); + if (columnToCheckIndex < 0 || columnToCheckIndex >= cacheValue.size()) { + return new byte[0]; + } + + final String valueFromColumnToCheck = cacheValue.get(columnToCheckIndex); + + if (valueFromColumnToCheck != null && valueFromColumnToCheck.equals(decodeConfig.getValueToCheck())) { + final int columnIfValueMatchedIndex = extractionNamespace.getColumnIndex(decodeConfig.getColumnIfValueMatched()); + if (columnIfValueMatchedIndex < 0) { + return new byte[0]; + } + String value = cacheValue.get(columnIfValueMatchedIndex); + return (value == null) ? new byte[0] : value.getBytes(); + } else { + final int columnIfValueNotMatchedIndex = extractionNamespace.getColumnIndex(decodeConfig.getColumnIfValueNotMatched()); + if (columnIfValueNotMatchedIndex < 0) { + return new byte[0]; + } + String value = cacheValue.get(columnIfValueNotMatchedIndex); + return (value == null) ? new byte[0] : value.getBytes(); + } + } + + @Override + public String getCacheSize(final MongoExtractionNamespace extractionNamespace, final Map> cache) { + if (!extractionNamespace.isCacheEnabled()) { + return String.valueOf(lookupService.getSize()); + } + return String.valueOf(cache.size()); + } + + @Override + public Long getLastUpdatedTime(final MongoExtractionNamespace extractionNamespace) { + if (!extractionNamespace.isCacheEnabled()) { + return lookupService.getLastUpdatedTime(new LookupService.LookupData(extractionNamespace)); + } + return extractionNamespace.getPreviousLastUpdateTime(); + } +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/NoopAuthHeaderFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/NoopAuthHeaderFactory.java new file mode 100644 index 000000000..7e04521b9 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/NoopAuthHeaderFactory.java @@ -0,0 +1,11 @@ +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; + +import java.util.Collections; +import java.util.Map; + +public class NoopAuthHeaderFactory implements AuthHeaderFactory { + @Override + public Map getAuthHeaders() { + return Collections.emptyMap(); + } +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java similarity index 88% rename from druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java rename to druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java index c3dc1b17e..7a5cc71fb 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaExtractionCacheManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java @@ -12,12 +12,10 @@ import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.emitter.service.ServiceMetricEvent; +import com.yahoo.maha.maha_druid_lookups.query.lookup.MongoLookupExtractor; import com.yahoo.maha.maha_druid_lookups.query.lookup.RocksDBLookupExtractor; import com.yahoo.maha.maha_druid_lookups.query.lookup.JDBCLookupExtractor; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.*; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.KafkaManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.RocksDBManager; @@ -36,16 +34,13 @@ /** * */ -public abstract class MahaExtractionCacheManager -{ - protected static class NamespaceImplData - { +public abstract class MahaNamespaceExtractionCacheManager { + protected static class NamespaceImplData { public NamespaceImplData( final ListenableFuture future, final ExtractionNamespace namespace, final String name - ) - { + ) { this.future = future; this.namespace = namespace; this.name = name; @@ -60,13 +55,13 @@ public NamespaceImplData( volatile String latestVersion = null; } - private static final Logger log = new Logger(MahaExtractionCacheManager.class); + private static final Logger log = new Logger(MahaNamespaceExtractionCacheManager.class); private final ListeningScheduledExecutorService listeningScheduledExecutorService; protected final ConcurrentMap implData = new ConcurrentHashMap<>(); protected final ConcurrentMap lookupExtractorMap = new ConcurrentHashMap<>(); protected final AtomicLong tasksStarted = new AtomicLong(0); protected final ServiceEmitter serviceEmitter; - private final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap; + private final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap; @Inject LookupService lookupService; @Inject @@ -76,12 +71,11 @@ public NamespaceImplData( @Inject ProtobufSchemaFactory protobufSchemaFactory; - public MahaExtractionCacheManager( + public MahaNamespaceExtractionCacheManager( Lifecycle lifecycle, final ServiceEmitter serviceEmitter, - final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap - ) - { + final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap + ) { this.listeningScheduledExecutorService = MoreExecutors.listeningDecorator( Executors.newScheduledThreadPool( 15, @@ -96,13 +90,11 @@ public MahaExtractionCacheManager( this.serviceEmitter = serviceEmitter; this.namespaceFunctionFactoryMap = namespaceFunctionFactoryMap; listeningScheduledExecutorService.scheduleAtFixedRate( - new Runnable() - { + new Runnable() { long priorTasksStarted = 0L; @Override - public void run() - { + public void run() { try { final long tasks = tasksStarted.get(); serviceEmitter.emit( @@ -111,8 +103,7 @@ public void run() ); priorTasksStarted = tasks; monitor(serviceEmitter); - } - catch (Exception e) { + } catch (Exception e) { log.error(e, "Error emitting namespace stats"); if (Thread.currentThread().isInterrupted()) { throw Throwables.propagate(e); @@ -130,19 +121,16 @@ public void run() * * @param serviceEmitter The emitter to emit to */ - protected void monitor(ServiceEmitter serviceEmitter) - { + protected void monitor(ServiceEmitter serviceEmitter) { // Noop by default } - protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws InterruptedException - { + protected boolean waitForServiceToEnd(long time, TimeUnit unit) throws InterruptedException { return listeningScheduledExecutorService.awaitTermination(time, unit); } - protected void updateNamespace(final String id, final String cacheId, final String newVersion) - { + protected void updateNamespace(final String id, final String cacheId, final String newVersion) { final NamespaceImplData namespaceDatum = implData.get(id); if (namespaceDatum == null) { // was removed @@ -160,8 +148,7 @@ protected void updateNamespace(final String id, final String cacheId, final Stri //swapAndClearCache(id, cacheId); namespaceDatum.latestVersion = newVersion; } - } - finally { + } finally { namespaceDatum.firstRun.countDown(); } } @@ -169,8 +156,7 @@ protected void updateNamespace(final String id, final String cacheId, final Stri // return value means actually delete or not public boolean checkedDelete( String namespaceName - ) - { + ) { final NamespaceImplData implDatum = implData.get(namespaceName); if (implDatum == null) { // Delete but we don't have it? @@ -184,8 +170,7 @@ public boolean checkedDelete( public boolean scheduleOrUpdate( final String id, ExtractionNamespace namespace - ) - { + ) { final NamespaceImplData implDatum = implData.get(id); if (implDatum == null) { // New, probably @@ -220,8 +205,7 @@ public boolean scheduleAndWait( final String id, ExtractionNamespace namespace, long waitForFirstRun - ) - { + ) { if (scheduleOrUpdate(id, namespace)) { log.debug("Scheduled new namespace [%s]: %s", id, namespace); } else { @@ -237,30 +221,25 @@ public boolean scheduleAndWait( boolean success = false; try { success = namespaceImplData.firstRun.await(waitForFirstRun, TimeUnit.MILLISECONDS); - } - catch (InterruptedException e) { + } catch (InterruptedException e) { log.error(e, "NamespaceLookupExtractorFactory[%s] - interrupted during start", id); } return success; } @GuardedBy("implDatum.changeLock") - private void cancelFuture(final NamespaceImplData implDatum) - { + private void cancelFuture(final NamespaceImplData implDatum) { final CountDownLatch latch = new CountDownLatch(1); final ListenableFuture future = implDatum.future; Futures.addCallback( - future, new FutureCallback() - { + future, new FutureCallback() { @Override - public void onSuccess(Object result) - { + public void onSuccess(Object result) { latch.countDown(); } @Override - public void onFailure(Throwable t) - { + public void onFailure(Throwable t) { // Expect CancellationException latch.countDown(); if (!(t instanceof CancellationException)) { @@ -272,8 +251,7 @@ public void onFailure(Throwable t) future.cancel(true); try { latch.await(); - } - catch (InterruptedException e) { + } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw Throwables.propagate(e); } @@ -281,8 +259,7 @@ public void onFailure(Throwable t) // Not thread safe @GuardedBy("implDatum.changeLock") - private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum) - { + private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum) { if (implDatum == null) { return false; } @@ -298,9 +275,8 @@ private boolean removeNamespaceLocalMetadata(final NamespaceImplData implDatum) } // Optimistic scheduling of updates to a namespace. - public ListenableFuture schedule(final String id, final T namespace) - { - final ExtractionNamespaceCacheFactory factory = (ExtractionNamespaceCacheFactory) + public ListenableFuture schedule(final String id, final T namespace) { + final ExtractionNamespaceCacheFactory factory = (ExtractionNamespaceCacheFactory) namespaceFunctionFactoryMap.get(namespace.getClass()); if (factory == null) { throw new ISE("Cannot find factory for namespace [%s]", namespace); @@ -313,10 +289,9 @@ public ListenableFuture schedule(final String protected synchronized ListenableFuture schedule( final String id, final T namespace, - final ExtractionNamespaceCacheFactory factory, + final ExtractionNamespaceCacheFactory factory, final String cacheId - ) - { + ) { log.info("Trying to update namespace [%s]", id); final NamespaceImplData implDatum = implData.get(id); if (implDatum != null) { @@ -332,11 +307,9 @@ protected synchronized ListenableFuture sched // Must be set before leader election occurs or else runnable will fail final AtomicReference implDataAtomicReference = new AtomicReference<>(null); - final Runnable command = new Runnable() - { + final Runnable command = new Runnable() { @Override - public void run() - { + public void run() { try { startLatch.await(); // wait for "election" to leadership or cancellation if (!Thread.currentThread().isInterrupted()) { @@ -358,16 +331,14 @@ public void run() log.debug("Version `%s` already exists, skipping updating cache", preVersion); } } - } - catch (Throwable t) { + } catch (Throwable t) { try { if (t instanceof InterruptedException) { log.info(t, "Namespace [%s] cancelled", id); } else { log.error(t, "Failed update namespace [%s]", namespace); } - } - catch (Exception e) { + } catch (Exception e) { t.addSuppressed(e); } if (Thread.currentThread().isInterrupted() || (t instanceof Error)) { @@ -404,17 +375,18 @@ public void run() log.debug("I own namespace [%s]", id); return future; } - } - finally { + } finally { startLatch.countDown(); } } private LookupExtractor getLookupExtractor(final ExtractionNamespace extractionNamespace, Map map) { - if(extractionNamespace instanceof JDBCExtractionNamespace) { - return new JDBCLookupExtractor((JDBCExtractionNamespace)extractionNamespace, map, lookupService); - } else if(extractionNamespace instanceof RocksDBExtractionNamespace) { + if (extractionNamespace instanceof JDBCExtractionNamespace) { + return new JDBCLookupExtractor((JDBCExtractionNamespace) extractionNamespace, map, lookupService); + } else if (extractionNamespace instanceof RocksDBExtractionNamespace) { return new RocksDBLookupExtractor((RocksDBExtractionNamespace) extractionNamespace, map, lookupService, rocksDBManager, kafkaManager, protobufSchemaFactory, serviceEmitter); + } else if (extractionNamespace instanceof MongoExtractionNamespace) { + return new MongoLookupExtractor((MongoExtractionNamespace) extractionNamespace, map, lookupService); } else { // return new MapLookupExtractor(map, false); return null; @@ -430,7 +402,6 @@ public LookupExtractor getLookupExtractor(final String id) { * * @param namespaceKey The namespace to swap the cache into * @param cacheKey The cacheKey that contains the data of interest - * * @return true if old data was cleared. False if no old data was found */ protected abstract boolean swapAndClearCache(String namespaceKey, String cacheKey); @@ -439,7 +410,6 @@ public LookupExtractor getLookupExtractor(final String id) { * Return a ConcurrentMap with the specified ID (either namespace's name or a cache key ID) * * @param namespaceOrCacheKey Either a namespace or cache key should be acceptable here. - * * @return A ConcurrentMap that is backed by the impl which implements this method. */ public abstract ConcurrentMap getCacheMap(String namespaceOrCacheKey); @@ -448,13 +418,10 @@ public LookupExtractor getLookupExtractor(final String id) { * Clears out resources used by the namespace such as threads. Implementations may override this and call super.delete(...) if they have resources of their own which need cleared. * * @param ns The namespace to be deleted - * * @return True if a deletion occurred, false if no deletion occurred. - * * @throws ISE if there is an error cancelling the namespace's future task */ - public boolean delete(final String ns) - { + public boolean delete(final String ns) { final NamespaceImplData implDatum = implData.get(ns); if (implDatum == null) { log.debug("Found no running cache for [%s]", ns); @@ -471,8 +438,7 @@ public boolean delete(final String ns) } } - public String getVersion(String namespace) - { + public String getVersion(String namespace) { if (namespace == null) { return null; } @@ -483,8 +449,7 @@ public String getVersion(String namespace) return implDatum.latestVersion; } - public Collection getKnownIDs() - { + public Collection getKnownIDs() { return implData.keySet(); } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/OnHeapMahaExtractionCacheManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/OnHeapMahaNamespaceExtractionCacheManager.java similarity index 93% rename from druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/OnHeapMahaExtractionCacheManager.java rename to druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/OnHeapMahaNamespaceExtractionCacheManager.java index 20ef6fbea..4f8a2e543 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/OnHeapMahaExtractionCacheManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/OnHeapMahaNamespaceExtractionCacheManager.java @@ -21,14 +21,14 @@ /** * */ -public class OnHeapMahaExtractionCacheManager extends MahaExtractionCacheManager +public class OnHeapMahaNamespaceExtractionCacheManager extends MahaNamespaceExtractionCacheManager { - private static final Logger LOG = new Logger(OnHeapMahaExtractionCacheManager.class); + private static final Logger LOG = new Logger(OnHeapMahaNamespaceExtractionCacheManager.class); private final ConcurrentMap> mapMap = new ConcurrentHashMap<>(); private final Striped nsLocks = Striped.lock(32); @Inject - public OnHeapMahaExtractionCacheManager( + public OnHeapMahaNamespaceExtractionCacheManager( final Lifecycle lifecycle, final ServiceEmitter emitter, final Map, ExtractionNamespaceCacheFactory> namespaceFunctionFactoryMap diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/FlatMultiValueDocumentProcessor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/FlatMultiValueDocumentProcessor.java new file mode 100644 index 000000000..4afc99a82 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/FlatMultiValueDocumentProcessor.java @@ -0,0 +1,115 @@ +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Objects; +import javax.validation.constraints.NotNull; + +import org.apache.commons.lang.StringUtils; +import org.bson.Document; +import org.bson.types.ObjectId; + +public class FlatMultiValueDocumentProcessor implements MongoDocumentProcessor { + + @JsonProperty + private final ArrayList columnList; + @JsonProperty + private final String primaryKeyColumn; + + @JsonCreator + public FlatMultiValueDocumentProcessor( + @NotNull @JsonProperty(value = "columnList", required = true) final ArrayList columnList, + @NotNull @JsonProperty(value = "primaryKeyColumn", required = true) final String primaryKeyColumn) { + this.columnList = columnList; + this.primaryKeyColumn = primaryKeyColumn; + Preconditions.checkArgument(StringUtils.isNotBlank(primaryKeyColumn), + "primaryKeyColumn cannot be blank or empty!"); + Preconditions.checkArgument(!columnList.isEmpty(), "columnList cannot be empty!"); + Preconditions.checkArgument(columnList.stream().noneMatch(StringUtils::isBlank), + "columnList cannot have blank columns in list!"); + Preconditions.checkArgument(columnList.stream().noneMatch(s -> s.equals(primaryKeyColumn)), + "columnList cannot have primaryKeyColumn in list!"); + } + + @Override + public ArrayList getColumnList() { + return columnList; + } + + @Override + public String getPrimaryKeyColumn() { + return primaryKeyColumn; + } + + private String asString(Object objectFromMongo) { + if (objectFromMongo instanceof String) { + return objectFromMongo.toString(); + } + if (objectFromMongo instanceof Integer) { + return ((Integer) objectFromMongo).toString(); + } + if (objectFromMongo instanceof Float) { + return ((Float) objectFromMongo).toString(); + } + if (objectFromMongo instanceof Double) { + return ((Double) objectFromMongo).toString(); + } + if (objectFromMongo instanceof Boolean) { + return ((Boolean) objectFromMongo).toString(); + } + if (objectFromMongo instanceof Date) { + return ((Date) objectFromMongo).toString(); + } + if (objectFromMongo instanceof ObjectId) { + return ((ObjectId) objectFromMongo).toHexString(); + } + throw new RuntimeException( + String.format("Unhanlded mongo object type : %s", objectFromMongo.getClass().getName())); + } + + @Override + public void process(Document document, LookupBuilder lookupBuilder) { + Object keyObject = document.get(primaryKeyColumn); + + if (keyObject != null) { + String keyValue = asString(keyObject); + List colValues = new ArrayList<>(columnList.size()); + for (String col : columnList) { + Object colObject = document.get(col); + String colValue = ""; + if (colObject != null) { + colValue = asString(colObject); + } + colValues.add(colValue); + } + lookupBuilder.add(keyValue, colValues); + } + } + + @Override + public String toString() { + return "FlatMultiValueDocumentProcessor{" + + "columnList=" + columnList + + ", primaryKeyColumn='" + primaryKeyColumn + '\'' + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + FlatMultiValueDocumentProcessor that = (FlatMultiValueDocumentProcessor) o; + return Objects.equals(getColumnList(), that.getColumnList()) && + Objects.equals(getPrimaryKeyColumn(), that.getPrimaryKeyColumn()); + } + + @Override + public int hashCode() { + return Objects.hash(getColumnList(), getPrimaryKeyColumn()); + } +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/LookupBuilder.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/LookupBuilder.java new file mode 100644 index 000000000..63e7d57f3 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/LookupBuilder.java @@ -0,0 +1,17 @@ +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity; + +import java.util.List; +import java.util.Map; + +public class LookupBuilder { + + private final Map> cache; + + public LookupBuilder(Map> cache) { + this.cache = cache; + } + + public void add(String key, List value) { + cache.put(key, value); + } +} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/MongoDocumentProcessor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/MongoDocumentProcessor.java new file mode 100644 index 000000000..77237323c --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/MongoDocumentProcessor.java @@ -0,0 +1,23 @@ +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import io.druid.guice.annotations.ExtensionPoint; + +import java.util.List; + +import org.bson.Document; + +@ExtensionPoint +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = FlatMultiValueDocumentProcessor.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "flatmultivalue", value = FlatMultiValueDocumentProcessor.class), +}) +public interface MongoDocumentProcessor { + void process(Document document, LookupBuilder lookupBuilder); + + List getColumnList(); + + String getPrimaryKeyColumn(); + +} diff --git a/druid-lookups/src/resources/META-INF/services/io.druid.initialization.DruidModule b/druid-lookups/src/main/resources/META-INF/services/io.druid.initialization.DruidModule similarity index 58% rename from druid-lookups/src/resources/META-INF/services/io.druid.initialization.DruidModule rename to druid-lookups/src/main/resources/META-INF/services/io.druid.initialization.DruidModule index 63b55aecd..0d2fcc0c7 100644 --- a/druid-lookups/src/resources/META-INF/services/io.druid.initialization.DruidModule +++ b/druid-lookups/src/main/resources/META-INF/services/io.druid.initialization.DruidModule @@ -1,2 +1,2 @@ com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.MahaNamespaceExtractionModule -com.yahoo.maha.query.aggregation.RoundingDoubleSumDruidModule +com.yahoo.maha.query.aggregation.RoundingDoubleSumDruidModule \ No newline at end of file diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractorTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractorTest.java index 32b64927d..989cf652f 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractorTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/JDBCLookupExtractorTest.java @@ -5,7 +5,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.RocksDBManager; import io.druid.metadata.MetadataStorageConnectorConfig; import org.joda.time.Period; import org.mockito.Mock; @@ -19,9 +18,6 @@ public class JDBCLookupExtractorTest { private static final String CONTROL_A_SEPARATOR = "\u0001"; private static final ObjectMapper objectMapper = new ObjectMapper(); - @Mock - RocksDBManager rocksDBManager; - @Mock LookupService lookupService; @@ -30,7 +26,7 @@ public void testBuildWhenDimensionValueIsEmpty() { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = - new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map map = new HashMap<>(); @@ -44,7 +40,7 @@ public void testBuildWhenDimensionValueIsNull() { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = - new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map map = new HashMap<>(); @@ -59,7 +55,7 @@ public void testBuildWhenCacheValueIsNull() throws Exception { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map map = new HashMap<>(); JDBCLookupExtractor JDBCLookupExtractor = new JDBCLookupExtractor(extractionNamespace, map, lookupService); @@ -76,7 +72,7 @@ public void testBuildWhenCacheValueIsNotNull() throws Exception { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); @@ -106,7 +102,7 @@ public void testBuildWhenInvalidValueColumn() throws Exception { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); @@ -124,7 +120,7 @@ public void testDimensionOverrideMap() throws Exception { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); @@ -155,7 +151,7 @@ public void testDecodeConfig() throws Exception { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = new JDBCExtractionNamespace( - metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); map.put("12345", Arrays.asList("12345", "my name", "USD", "ON")); diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java index 6e2f30e8e..8ff4212d6 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java @@ -32,7 +32,7 @@ public void testWhenCacheValueIsEmpty() { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = - new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); @@ -48,9 +48,9 @@ public void testWhenCacheValueIsEmpty() { when(lrm.get(anyString())).thenReturn(lefc); MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, true)); - fn.apply("123"); - Assert.assertEquals(fn.cache.getIfPresent("123"), "{\"dimension\":\"123\",\"valueColumn\":\"status\",\"decodeConfig\":null,\"dimensionOverrideMap\":null}"); - verify(fn, times(1)).getSerializedLookupQueryElement(anyString()); + Assert.assertNull(fn.cache); + Assert.assertEquals(fn.apply("123"), "ON"); + Assert.assertEquals(fn.cache.getIfPresent("123"), "ON"); } @Test @@ -58,7 +58,7 @@ public void testWhenCacheValueIsPresent() { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = - new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); @@ -75,11 +75,10 @@ public void testWhenCacheValueIsPresent() { MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, true)); - fn.ensureCache().put("123", "{\"dimension\":\"123\",\"valueColumn\":\"status\",\"decodeConfig\":null,\"dimensionOverrideMap\":null}"); + fn.ensureCache().put("123", "hola"); - fn.apply("123"); - Assert.assertEquals(fn.cache.getIfPresent("123"), "{\"dimension\":\"123\",\"valueColumn\":\"status\",\"decodeConfig\":null,\"dimensionOverrideMap\":null}"); - verify(fn, times(0)).getSerializedLookupQueryElement(anyString()); + Assert.assertEquals(fn.apply("123"), "hola"); + Assert.assertEquals(fn.cache.getIfPresent("123"), "hola"); } @Test @@ -87,7 +86,7 @@ public void testWhenUseQueryLevelCacheIsFalse() { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = - new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); @@ -104,9 +103,9 @@ public void testWhenUseQueryLevelCacheIsFalse() { MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, false)); - fn.apply("123"); Assert.assertNull(fn.cache); - verify(fn, times(1)).getSerializedLookupQueryElement(anyString()); + Assert.assertEquals(fn.apply("123"), "ON"); + Assert.assertNull(fn.cache); } @Test @@ -114,7 +113,7 @@ public void testWhenNullValueIsPresent() { MetadataStorageConnectorConfig metadataStorageConnectorConfig = new MetadataStorageConnectorConfig(); JDBCExtractionNamespace extractionNamespace = - new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id","name","currency","status")), + new JDBCExtractionNamespace(metadataStorageConnectorConfig, "advertiser", new ArrayList<>(Arrays.asList("id", "name", "currency", "status")), "id", "", new Period(), true, "advertiser_lookup"); Map> map = new HashMap<>(); diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractorTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractorTest.java new file mode 100644 index 000000000..bd4206806 --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MongoLookupExtractorTest.java @@ -0,0 +1,230 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.query.lookup; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.mongodb.MongoClient; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoStorageConnectorConfig; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.LookupService; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.FlatMultiValueDocumentProcessor; +import de.bwaldvogel.mongo.MongoServer; +import de.bwaldvogel.mongo.backend.memory.MemoryBackend; +import org.joda.time.Period; +import org.mockito.Mock; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.net.InetSocketAddress; +import java.util.*; + +public class MongoLookupExtractorTest { + + private static final String CONTROL_A_SEPARATOR = "\u0001"; + private static final ObjectMapper objectMapper = new ObjectMapper(); + + @Mock + LookupService lookupService; + + MongoServer mongoServer; + + MongoStorageConnectorConfig mongoStorageConnectorConfig; + + MongoClient mongoClient; + + @BeforeClass + public void setup() throws Exception { + mongoServer = new MongoServer(new MemoryBackend()); + InetSocketAddress serverAddress = mongoServer.bind(); + String jsonConfig = String.format("{\n" + + "\t\"hosts\": \"%s:%s\",\n" + + "\t\"dbName\": \"mydb\",\n" + + "\t\"user\": \"test-user\",\n" + + "\t\"password\": {\n" + + "\t\t\"type\": \"default\",\n" + + "\t\t\"password\": \"mypassword\"\n" + + "\t},\n" + + "\t\"clientOptions\": {\n" + + "\t\t\"connectionsPerHost\": \"3\",\n" + + "\t\t\"socketTimeout\": \"10000\",\n" + + "\t\t\"connectTimeout\": \"2000\"\n" + + "\t}\n" + + "}", serverAddress.getHostString(), serverAddress.getPort()); + mongoStorageConnectorConfig = objectMapper.readValue(jsonConfig, MongoStorageConnectorConfig.class); + mongoClient = mongoStorageConnectorConfig.getMongoClient(); + } + + @AfterClass + public void cleanup() throws Exception { + mongoClient.close(); + mongoServer.shutdownNow(); + } + + @Test + public void testBuildWhenDimensionValueIsEmpty() { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + + Map map = new HashMap<>(); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + String lookupValue = MongoLookupExtractor.apply(""); + Assert.assertNull(lookupValue); + } + + @Test + public void testBuildWhenDimensionValueIsNull() { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + + Map map = new HashMap<>(); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + String lookupValue = MongoLookupExtractor.apply(null); + Assert.assertNull(lookupValue); + } + + @Test + public void testBuildWhenCacheValueIsNull() throws Exception { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map map = new HashMap<>(); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); + mahaLookupQueryElement1.setDimension("12345"); + mahaLookupQueryElement1.setValueColumn("name"); + String lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + Assert.assertNull(lookupValue); + } + + @Test + public void testBuildWhenCacheValueIsNotNull() throws Exception { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); + mahaLookupQueryElement1.setDimension("12345"); + mahaLookupQueryElement1.setValueColumn("status"); + String lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + Assert.assertEquals(lookupValue, "ON"); + + MahaLookupQueryElement mahaLookupQueryElement2 = new MahaLookupQueryElement(); + mahaLookupQueryElement2.setDimension("12345"); + mahaLookupQueryElement2.setValueColumn("name"); + lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); + Assert.assertEquals(lookupValue, "my name"); + + MahaLookupQueryElement mahaLookupQueryElement3 = new MahaLookupQueryElement(); + mahaLookupQueryElement3.setDimension("12345"); + mahaLookupQueryElement3.setValueColumn("currency"); + lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement3)); + Assert.assertEquals(lookupValue, "USD"); + } + + @Test + public void testBuildWhenInvalidValueColumn() throws Exception { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); + mahaLookupQueryElement1.setDimension("12345"); + mahaLookupQueryElement1.setValueColumn("booking_country"); + String lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + Assert.assertNull(lookupValue); + } + + @Test + public void testDimensionOverrideMap() throws Exception { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + Map dimensionOverrideMap = new HashMap<>(); + dimensionOverrideMap.put("12345", "something-12345"); + dimensionOverrideMap.put("6789", "something-6789"); + dimensionOverrideMap.put("", "Unknown"); + + MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); + mahaLookupQueryElement1.setDimension("12345"); + mahaLookupQueryElement1.setValueColumn("name"); + mahaLookupQueryElement1.setDimensionOverrideMap(dimensionOverrideMap); + String lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + Assert.assertEquals(lookupValue, "something-12345"); + + MahaLookupQueryElement mahaLookupQueryElement2 = new MahaLookupQueryElement(); + mahaLookupQueryElement2.setDimension(""); + mahaLookupQueryElement2.setValueColumn("name"); + mahaLookupQueryElement2.setDimensionOverrideMap(dimensionOverrideMap); + lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); + Assert.assertEquals(lookupValue, "Unknown"); + } + + @Test + public void testDecodeConfig() throws Exception { + + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + MongoLookupExtractor MongoLookupExtractor = new MongoLookupExtractor(extractionNamespace, map, lookupService); + + Map dimensionOverrideMap = new HashMap<>(); + dimensionOverrideMap.put("123", "something-123"); + dimensionOverrideMap.put("6789", "something-6789"); + dimensionOverrideMap.put("", "Unknown"); + + MahaLookupQueryElement mahaLookupQueryElement1 = new MahaLookupQueryElement(); + mahaLookupQueryElement1.setDimension("12345"); + mahaLookupQueryElement1.setValueColumn("name"); + mahaLookupQueryElement1.setDimensionOverrideMap(dimensionOverrideMap); + DecodeConfig decodeConfig1 = new DecodeConfig(); + decodeConfig1.setColumnToCheck("name"); + decodeConfig1.setValueToCheck("my name"); + decodeConfig1.setColumnIfValueMatched("currency"); + decodeConfig1.setColumnIfValueNotMatched("status"); + mahaLookupQueryElement1.setDecodeConfig(decodeConfig1); + String lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement1)); + Assert.assertEquals(lookupValue, "USD"); + + + MahaLookupQueryElement mahaLookupQueryElement2 = new MahaLookupQueryElement(); + mahaLookupQueryElement2.setDimension("12345"); + mahaLookupQueryElement2.setValueColumn("name"); + mahaLookupQueryElement2.setDimensionOverrideMap(dimensionOverrideMap); + DecodeConfig decodeConfig2 = new DecodeConfig(); + decodeConfig2.setColumnToCheck("name"); + decodeConfig2.setValueToCheck("my unknown name"); + decodeConfig2.setColumnIfValueMatched("currency"); + decodeConfig2.setColumnIfValueNotMatched("status"); + mahaLookupQueryElement2.setDecodeConfig(decodeConfig2); + lookupValue = MongoLookupExtractor.apply(objectMapper.writeValueAsString(mahaLookupQueryElement2)); + Assert.assertEquals(lookupValue, "ON"); + + } + +} \ No newline at end of file diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespaceTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespaceTest.java new file mode 100644 index 000000000..5651b87fe --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoExtractionNamespaceTest.java @@ -0,0 +1,63 @@ +package com.yahoo.maha.maha_druid_lookups.query.lookup.namespace; + +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.deser.DurationDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.PeriodDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.key.DateTimeKeyDeserializer; +import com.google.common.collect.Lists; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + +public class MongoExtractionNamespaceTest { + private static final ObjectMapper objectMapper = new ObjectMapper(); + + + @BeforeClass + void setup() { + SimpleModule module = new SimpleModule(); + module.addKeyDeserializer(DateTime.class, new DateTimeKeyDeserializer()); + module.addSerializer(DateTime.class, ToStringSerializer.instance); + module.addSerializer(Interval.class, ToStringSerializer.instance); + JsonDeserializer periodDeserializer = new PeriodDeserializer(); + module.addDeserializer(Period.class, (JsonDeserializer) periodDeserializer); + module.addSerializer(Period.class, ToStringSerializer.instance); + module.addDeserializer(Duration.class, new DurationDeserializer()); + module.addSerializer(Duration.class, ToStringSerializer.instance); + + objectMapper.registerModule(module); + + Period period = Period.seconds(30); + System.out.println(period); + } + + @Test + public void successfullyDeserializeFullNamespaceFromJSON() throws Exception { + MongoExtractionNamespace namespace = objectMapper + .readValue(ClassLoader.getSystemClassLoader().getResourceAsStream("mongo_extraction_namespace.json") + , MongoExtractionNamespace.class); + assertEquals(namespace.getConnectorConfig().getHosts(), "localhost:51240"); + assertEquals(namespace.getConnectorConfig().getDbName(), "advertiser"); + assertEquals(namespace.getConnectorConfig().getUser(), "test-user"); + assertEquals(namespace.getConnectorConfig().getPassword(), "mypassword"); + assertEquals(namespace.getConnectorConfig().getClientProperties().getProperty("socketTimeout").toString(), "30000"); + assertEquals(namespace.getCollectionName(), "advertiser"); + assertEquals(namespace.getTsColumn(), "updated_at"); + assertEquals(namespace.isTsColumnEpochInteger(), true); + assertEquals(namespace.getPollMs(), 30000); + assertEquals(namespace.isCacheEnabled(), true); + assertEquals(namespace.getLookupName(), "advertiser_lookup"); + assertEquals(namespace.getDocumentProcessor().getColumnList(), Lists.newArrayList("name", "currency", "status")); + assertEquals(namespace.getDocumentProcessor().getPrimaryKeyColumn(), "_id"); + assertEquals(namespace.getMongoClientRetryCount(), 3); + + } +} diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfigTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfigTest.java new file mode 100644 index 000000000..bfaccf020 --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/MongoStorageConnectorConfigTest.java @@ -0,0 +1,88 @@ +package com.yahoo.maha.maha_druid_lookups.query.lookup.namespace; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Joiner; +import com.mongodb.MongoClientOptions; +import org.testng.annotations.Test; + +import static org.testng.Assert.*; + +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MongoStorageConnectorConfigTest { + + @Test + public void successFullyDeserializeFullConfig() throws Exception { + final ObjectMapper objectMapper = new ObjectMapper(); + Map integerMap = new HashMap<>(); + Map boolMap = new HashMap<>(); + + int i = 2000; + for (String prop : MongoStorageConnectorConfig.INT_PROPERTIES) { + integerMap.put(prop, i--); + } + + for (String prop : MongoStorageConnectorConfig.BOOL_PROPERTIES) { + boolMap.put(prop, true); + } + + Joiner commaJoiner = Joiner.on(","); + + List props = new ArrayList<>(); + integerMap.entrySet().stream().forEach(entry -> props.add(String.format("\"%s\":\"%s\"", entry.getKey(), entry.getValue()))); + boolMap.entrySet().stream().forEach(entry -> props.add(String.format("\"%s\":\"%s\"", entry.getKey(), entry.getValue()))); + + String clientOptionsJson = commaJoiner.join(props); + + String jsonConfig = String.format("{\n" + + "\t\"hosts\": \"host1:2001,host2:2002,host3:2003\",\n" + + "\t\"dbName\": \"mydb\",\n" + + "\t\"user\": \"test-user\",\n" + + "\t\"password\": {\n" + + "\t\t\"type\": \"default\",\n" + + "\t\t\"password\": \"mypassword\"\n" + + "\t},\n" + + "\t\"clientOptions\": {\n%s" + + "\t}\n" + + "}", clientOptionsJson); + + MongoStorageConnectorConfig read = objectMapper.readValue(jsonConfig, MongoStorageConnectorConfig.class); + assertEquals(read.getHosts(), "host1:2001,host2:2002,host3:2003"); + assertEquals(read.getDbName(), "mydb"); + assertEquals(read.getUser(), "test-user"); + assertEquals(read.getPassword(), "mypassword"); + MongoClientOptions options = read.getMongoClientOptions(); + + Class clazz = MongoClientOptions.class; + for (String p : MongoStorageConnectorConfig.INT_PROPERTIES) { + Integer value = integerMap.get(p); + if (value != null) { + try { + String methodName = String.format("get%s%s", p.toUpperCase().charAt(0), p.substring(1)); + Method m = clazz.getMethod(methodName); + assertEquals(m.invoke(options), value); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to parse int for property : %s=%s", p, value), e); + } + } + } + for (String p : MongoStorageConnectorConfig.BOOL_PROPERTIES) { + Boolean value = boolMap.get(p); + if (value != null) { + try { + String methodName = String.format("is%s%s", p.toUpperCase().charAt(0), p.substring(1)); + Method m = clazz.getMethod(methodName); + assertEquals(m.invoke(options), value); + } catch (Exception e) { + throw new RuntimeException( + String.format("Failed to parse boolean for property : %s=%s", p, value), e); + } + } + } + } +} diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java new file mode 100644 index 000000000..359ce47b6 --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java @@ -0,0 +1,51 @@ +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaNamespaceExtractionCacheManager; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import org.testng.Assert; +import org.testng.annotations.Test; + +import java.util.Properties; + +public class MahaNamespaceExtractionModuleTest { + @Test + public void testInjection() + { + Injector injector = GuiceInjectors.makeStartupInjector(); + final Properties properties = injector.getInstance(Properties.class); + properties.put(MahaNamespaceExtractionModule.TYPE_PREFIX, "onHeap"); + properties.put(String.format("%s.authHeaderFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.NoopAuthHeaderFactory"); + properties.put(String.format("%s.schemaFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.DefaultProtobufSchemaFactory"); + + injector = Initialization.makeInjectorWithModules( + injector, + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bindInstance( + binder, + Key.get(DruidNode.class, Self.class), + new DruidNode("test-inject", null, null, null, true, false) + ); + } + } + ) + ); + final MahaNamespaceExtractionCacheManager manager = injector.getInstance(MahaNamespaceExtractionCacheManager.class); + Assert.assertEquals(OnHeapMahaNamespaceExtractionCacheManager.class, manager.getClass()); + } +} diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResourceTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResourceTest.java index 44f6e3bab..bf68e6124 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResourceTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResourceTest.java @@ -6,7 +6,7 @@ import com.metamx.emitter.service.ServiceEmitter; import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaNamespaceExtractionCacheManager; import io.druid.metadata.MetadataStorageConnectorConfig; import org.joda.time.Period; import org.testng.Assert; @@ -33,7 +33,7 @@ public class MahaNamespacesCacheResourceTest { @Test public void testGetCacheValue() throws Exception { - OnHeapMahaExtractionCacheManager cacheManager = mock(OnHeapMahaExtractionCacheManager.class); + OnHeapMahaNamespaceExtractionCacheManager cacheManager = mock(OnHeapMahaNamespaceExtractionCacheManager.class); ServiceEmitter serviceEmitter = mock(ServiceEmitter.class); HttpServletRequest httpServletRequest = mock(HttpServletRequest.class); MahaNamespacesCacheResource resource = new MahaNamespacesCacheResource(cacheManager, serviceEmitter); @@ -65,7 +65,7 @@ public void testGetCacheValue() throws Exception { @Test public void testGetCacheValueWhenDecodeConfigPresent() throws Exception { - OnHeapMahaExtractionCacheManager cacheManager = mock(OnHeapMahaExtractionCacheManager.class); + OnHeapMahaNamespaceExtractionCacheManager cacheManager = mock(OnHeapMahaNamespaceExtractionCacheManager.class); ServiceEmitter serviceEmitter = mock(ServiceEmitter.class); HttpServletRequest httpServletRequest = mock(HttpServletRequest.class); MahaNamespacesCacheResource resource = new MahaNamespacesCacheResource(cacheManager, serviceEmitter); diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java new file mode 100644 index 000000000..def96d8ea --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java @@ -0,0 +1,228 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.metamx.emitter.service.ServiceEmitter; +import com.mongodb.BasicDBObject; +import com.mongodb.MongoClient; +import com.mongodb.client.FindIterable; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.Projections; +import com.mongodb.client.result.UpdateResult; +import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoStorageConnectorConfig; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.FlatMultiValueDocumentProcessor; +import de.bwaldvogel.mongo.MongoServer; +import de.bwaldvogel.mongo.backend.memory.MemoryBackend; +import io.druid.metadata.MetadataStorageConnectorConfig; +import org.bson.Document; +import org.bson.types.ObjectId; +import org.joda.time.Period; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.testng.Assert; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.BeforeTest; +import org.testng.annotations.Test; + +import java.net.InetSocketAddress; +import java.util.*; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; + +public class MongoExtractionNamespaceCacheFactoryTest { + + private static final ObjectMapper objectMapper = new ObjectMapper(); + MongoExtractionNamespaceCacheFactory obj = new MongoExtractionNamespaceCacheFactory(); + + @Mock + ServiceEmitter serviceEmitter; + + @Mock + LookupService lookupService; + + MongoServer mongoServer; + + MongoStorageConnectorConfig mongoStorageConnectorConfig; + + MongoClient mongoClient; + + private static final String DEFAULT_COLLECTION = "advertiser"; + + @BeforeClass + public void setup() throws Exception { + mongoServer = new MongoServer(new MemoryBackend()); + InetSocketAddress serverAddress = mongoServer.bind(); + String jsonConfig = String.format("{\n" + + "\t\"hosts\": \"%s:%s\",\n" + + "\t\"dbName\": \"mydb\",\n" + + "\t\"clientOptions\": {\n" + + "\t\t\"connectionsPerHost\": \"3\",\n" + + "\t\t\"socketTimeout\": \"10000\",\n" + + "\t\t\"connectTimeout\": \"2000\"\n" + + "\t}\n" + + "}", serverAddress.getHostString(), serverAddress.getPort()); + mongoStorageConnectorConfig = objectMapper.readValue(jsonConfig, MongoStorageConnectorConfig.class); + mongoClient = mongoStorageConnectorConfig.getMongoClient(); + createTestData(); + } + + private void createTestData() throws Exception { + MongoClient localMongoClient = mongoStorageConnectorConfig.getMongoClient(); + MongoDatabase database = localMongoClient.getDatabase(mongoStorageConnectorConfig.getDbName()); + database.createCollection(DEFAULT_COLLECTION); + MongoCollection collection = database.getCollection(DEFAULT_COLLECTION); + JsonNode node = objectMapper.readValue( + ClassLoader.getSystemClassLoader().getResourceAsStream("mongo_advertiser.json"), JsonNode.class); + for (int i = 0; i < node.size(); i++) { + JsonNode elem = node.get(i); + String json = objectMapper.writeValueAsString(elem); + Document d = Document.parse(json); + collection.insertOne(d); + } + for (Document d : collection.find()) { + System.out.println(d.toJson()); + } + localMongoClient.close(); + } + + private void updateTestData(String jsonResource) throws Exception { + MongoClient localMongoClient = mongoStorageConnectorConfig.getMongoClient(); + MongoDatabase database = localMongoClient.getDatabase(mongoStorageConnectorConfig.getDbName()); + MongoCollection collection = database.getCollection(DEFAULT_COLLECTION); + JsonNode node = objectMapper.readValue( + ClassLoader.getSystemClassLoader().getResourceAsStream(jsonResource), JsonNode.class); + for (int i = 0; i < node.size(); i++) { + JsonNode elem = node.get(i); + String id = elem.get("_id").get("$oid").asText(); + String json = objectMapper.writeValueAsString(elem); + Document d = Document.parse(json); + UpdateResult result = collection.replaceOne(new BasicDBObject("_id", new ObjectId(id)), d); + if (result.getMatchedCount() <= 0) { + collection.insertOne(d); + } + } + for (Document d : collection.find()) { + System.out.println(d.toJson()); + } + localMongoClient.close(); + } + + @AfterClass + public void cleanup() throws Exception { + mongoClient.close(); + mongoServer.shutdownNow(); + } + + @BeforeTest + public void setUp() { + MockitoAnnotations.initMocks(this); + obj.emitter = serviceEmitter; + obj.lookupService = lookupService; + } + + @Test + public void testGetCacheValueWhenKeyPresent() throws Exception { + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + Assert.assertEquals(obj.getCacheValue(extractionNamespace, map, "12345", "name", Optional.empty()), "my name".getBytes()); + } + + @Test + public void testGetCacheValueWhenKeyNotPresent() throws Exception { + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + Assert.assertEquals(obj.getCacheValue(extractionNamespace, map, "6789", "name", Optional.empty()), "".getBytes()); + } + + @Test + public void testGetCacheValueWhenKeyPresentButValueColumnNotPresent() throws Exception { + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + Assert.assertEquals(obj.getCacheValue(extractionNamespace, map, "6789", "blah", Optional.empty()), "".getBytes()); + } + + @Test + public void testGetCacheValueWithDecodeConfig() throws Exception { + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Map> map = new HashMap<>(); + map.put("12345", Arrays.asList("my name", "USD", "ON")); + DecodeConfig decodeConfig1 = new DecodeConfig(); + decodeConfig1.setColumnToCheck("name"); + decodeConfig1.setValueToCheck("my name"); + decodeConfig1.setColumnIfValueMatched("currency"); + decodeConfig1.setColumnIfValueNotMatched("status"); + Assert.assertEquals(obj.getCacheValue(extractionNamespace, map, "12345", "name", Optional.of(decodeConfig1)), "USD".getBytes()); + + DecodeConfig decodeConfig2 = new DecodeConfig(); + decodeConfig2.setColumnToCheck("name"); + decodeConfig2.setValueToCheck("my unknown name"); + decodeConfig2.setColumnIfValueMatched("currency"); + decodeConfig2.setColumnIfValueNotMatched("status"); + Assert.assertEquals(obj.getCacheValue(extractionNamespace, map, "12345", "name", Optional.of(decodeConfig2)), "ON".getBytes()); + } + + @Test + public void testGetCachePopulator() throws Exception { + Map> cache = new ConcurrentHashMap<>(); + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Callable command = obj.getCachePopulator(extractionNamespace.getLookupName(), extractionNamespace, null, cache); + String version = command.call(); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41080")); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41081")); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41082")); + Assert.assertEquals(version, "1543652000"); + + } + + @Test + public void testGetCachePopulatorWithUpdatesAndAdditions() throws Exception { + Map> cache = new ConcurrentHashMap<>(); + MongoExtractionNamespace extractionNamespace = + new MongoExtractionNamespace(mongoStorageConnectorConfig, "advertiser" + , "updated_at", true, new Period(), true, "advertiser_lookup" + , new FlatMultiValueDocumentProcessor(new ArrayList<>(Arrays.asList("name", "currency", "status")), "_id"), null); + Callable command = obj.getCachePopulator(extractionNamespace.getLookupName(), extractionNamespace, null, cache); + String version = command.call(); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41080")); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41081")); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41082")); + Assert.assertEquals(version, "1543652000"); + Assert.assertEquals(cache.get("5ad10906fc7b6ecac8d41080").get(0), "123"); + Assert.assertEquals(cache.get("5ad10906fc7b6ecac8d41081").get(2), "ON"); + updateTestData("mongo_advertiser_update.json"); + version = command.call(); + Assert.assertEquals(version, "1543652001"); + Assert.assertEquals(cache.get("5ad10906fc7b6ecac8d41080").get(0), "advertiser3"); + Assert.assertEquals(cache.get("5ad10906fc7b6ecac8d41081").get(2), "OFF"); + updateTestData("mongo_advertiser_addition.json"); + Date currDate = new Date(); + version = command.call(); + Assert.assertTrue(Integer.parseInt(version) >= currDate.getTime() / 1000, String.format("%s not > %d", version, currDate.getTime() / 1000)); + Assert.assertTrue(cache.containsKey("5ad10906fc7b6ecac8d41083")); + } +} diff --git a/druid-lookups/src/test/resources/mongo_advertiser.json b/druid-lookups/src/test/resources/mongo_advertiser.json new file mode 100644 index 000000000..9b1e78af2 --- /dev/null +++ b/druid-lookups/src/test/resources/mongo_advertiser.json @@ -0,0 +1,29 @@ +[ + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41080" + }, + "name": "123", + "currency": "USD", + "status": "ON", + "updated_at": 1543651008 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41081" + }, + "name": "advertiser1", + "currency": "INR", + "status": "ON", + "updated_at": 1543652000 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41082" + }, + "name": "advertiser2", + "currency": "CAD", + "status": "OFF", + "updated_at": 1533651008 + } +] \ No newline at end of file diff --git a/druid-lookups/src/test/resources/mongo_advertiser_addition.json b/druid-lookups/src/test/resources/mongo_advertiser_addition.json new file mode 100644 index 000000000..50af9b178 --- /dev/null +++ b/druid-lookups/src/test/resources/mongo_advertiser_addition.json @@ -0,0 +1,38 @@ +[ + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41080" + }, + "name": "advertiser3", + "currency": "USD", + "status": "ON", + "updated_at": 1543652001 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41081" + }, + "name": "advertiser1", + "currency": "INR", + "status": "OFF", + "updated_at": 1543652001 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41082" + }, + "name": "advertiser2", + "currency": "CAD", + "status": "OFF", + "updated_at": 1533651008 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41083" + }, + "name": "advertiser4", + "currency": "AED", + "status": "ON", + "updated_at": 1633651008 + } +] \ No newline at end of file diff --git a/druid-lookups/src/test/resources/mongo_advertiser_update.json b/druid-lookups/src/test/resources/mongo_advertiser_update.json new file mode 100644 index 000000000..279d4538a --- /dev/null +++ b/druid-lookups/src/test/resources/mongo_advertiser_update.json @@ -0,0 +1,29 @@ +[ + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41080" + }, + "name": "advertiser3", + "currency": "USD", + "status": "ON", + "updated_at": 1543652001 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41081" + }, + "name": "advertiser1", + "currency": "INR", + "status": "OFF", + "updated_at": 1543652001 + }, + { + "_id": { + "$oid": "5ad10906fc7b6ecac8d41082" + }, + "name": "advertiser2", + "currency": "CAD", + "status": "OFF", + "updated_at": 1533651008 + } +] \ No newline at end of file diff --git a/druid-lookups/src/test/resources/mongo_extraction_namespace.json b/druid-lookups/src/test/resources/mongo_extraction_namespace.json new file mode 100644 index 000000000..1fa25ce97 --- /dev/null +++ b/druid-lookups/src/test/resources/mongo_extraction_namespace.json @@ -0,0 +1,31 @@ +{ + "type": "mahamongo", + "connectorConfig": { + "hosts": "localhost:51240", + "dbName": "advertiser", + "user": "test-user", + "password": { + "type": "default", + "password": "mypassword" + }, + "clientOptions": { + "socketTimeout": 30000 + } + }, + "collectionName": "advertiser", + "tsColumn": "updated_at", + "tsColumnEpochInteger": true, + "pollPeriod": "PT30S", + "cacheEnabled": true, + "lookupName": "advertiser_lookup", + "documentProcessor": { + "type": "flatmultivalue", + "columnList": [ + "name", + "currency", + "status" + ], + "primaryKeyColumn": "_id" + }, + "mongoClientRetryCount": 3 +} \ No newline at end of file diff --git a/pom.xml b/pom.xml index e44f19fdc..43f504ef9 100644 --- a/pom.xml +++ b/pom.xml @@ -63,6 +63,7 @@ 1.4.1 4.0.1 2.5.4 + 3.3.0 From 557fe93db25b1ed474ceca1f9ca59a5de9639674 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 16:25:26 -0800 Subject: [PATCH 03/15] Add tests and minor fixes --- .../lookup/MahaLookupExtractorFactory.java | 6 +- .../MahaRegisteredLookupExtractionFn.java | 10 +- .../lookup/namespace/ExtractionNamespace.java | 3 +- .../MahaNamespaceExtractionConfig.java | 6 +- .../MahaNamespaceExtractionModule.java | 9 +- .../lookup/namespace/RocksDBManager.java | 24 +-- .../MahaNamespaceExtractionCacheManager.java | 9 +- .../maha_druid_lookups/TestMongoServer.java | 76 +++++++++ .../MahaLookupExtractionFactoryTest.java | 156 ++++++++++++++++++ .../MahaRegisteredLookupExtractionFnTest.java | 8 +- .../MahaNamespaceExtractionModuleTest.java | 7 +- ...goExtractionNamespaceCacheFactoryTest.java | 57 +------ .../maha_lookup_extraction_factory.json | 34 ++++ .../maha_registered_lookup_extraction_fn.json | 7 + 14 files changed, 326 insertions(+), 86 deletions(-) create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/TestMongoServer.java create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java create mode 100644 druid-lookups/src/test/resources/maha_lookup_extraction_factory.json create mode 100644 druid-lookups/src/test/resources/maha_registered_lookup_extraction_fn.json diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java index 48cd1736f..84f201419 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractorFactory.java @@ -49,9 +49,9 @@ public class MahaLookupExtractorFactory implements LookupExtractorFactory @JsonCreator public MahaLookupExtractorFactory( - @JsonProperty("extractionNamespace") ExtractionNamespace extractionNamespace, - @JsonProperty("firstCacheTimeout") long firstCacheTimeout, - @JsonProperty("injective") boolean injective, + @JsonProperty(value = "extractionNamespace") ExtractionNamespace extractionNamespace, + @JsonProperty(value = "firstCacheTimeout") long firstCacheTimeout, //amount of time to wait for lookup to load + @JsonProperty(value = "injective") boolean injective, //true if there is one to one mapping from key to value columns @JacksonInject final MahaNamespaceExtractionCacheManager manager ) { diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java index 69929eee7..26fe664a6 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFn.java @@ -17,10 +17,12 @@ import io.druid.query.lookup.LookupReferencesManager; import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Map; +import java.util.Objects; import java.util.concurrent.TimeUnit; import com.google.common.base.Strings; @@ -32,7 +34,6 @@ public class MahaRegisteredLookupExtractionFn implements ExtractionFn { private volatile MahaLookupExtractionFn delegate = null; private final Object delegateLock = new Object(); private final LookupReferencesManager manager; - private final ObjectMapper objectMapper; private final String lookup; private final boolean retainMissingValue; private final String replaceMissingValueWith; @@ -48,20 +49,19 @@ public class MahaRegisteredLookupExtractionFn implements ExtractionFn { @JsonCreator public MahaRegisteredLookupExtractionFn( @JacksonInject LookupReferencesManager manager, - @JacksonInject ObjectMapper objectMapper, @JsonProperty("lookup") String lookup, @JsonProperty("retainMissingValue") final boolean retainMissingValue, @Nullable @JsonProperty("replaceMissingValueWith") final String replaceMissingValueWith, @JsonProperty("injective") final boolean injective, @JsonProperty("optimize") Boolean optimize, - @Nullable @JsonProperty("valueColumn") String valueColumn, + @NotNull @JsonProperty("valueColumn") String valueColumn, @Nullable @JsonProperty("decode") DecodeConfig decodeConfig, @Nullable @JsonProperty("dimensionOverrideMap") Map dimensionOverrideMap, @Nullable @JsonProperty("useQueryLevelCache") Boolean useQueryLevelCache ) { Preconditions.checkArgument(lookup != null, "`lookup` required"); + Preconditions.checkArgument(valueColumn != null, "`valueColumn` required"); this.manager = manager; - this.objectMapper = objectMapper; this.replaceMissingValueWith = replaceMissingValueWith; this.retainMissingValue = retainMissingValue; this.injective = injective; @@ -151,7 +151,7 @@ public byte[] getCacheKey() { @Override public String apply(Object value) { - return ensureDelegate().apply(value); + return apply(Objects.toString(value, null)); } @Override diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java index 6ffa7972c..1b41a9d7c 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java @@ -9,7 +9,8 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "mahajdbc", value = JDBCExtractionNamespace.class), - @JsonSubTypes.Type(name = "maharocksdb", value = RocksDBExtractionNamespace.class) + @JsonSubTypes.Type(name = "maharocksdb", value = RocksDBExtractionNamespace.class), + @JsonSubTypes.Type(name = "mahamongo", value = MongoExtractionNamespace.class) }) public interface ExtractionNamespace { diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java index 76f7bb1a5..57eace0e7 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionConfig.java @@ -10,12 +10,12 @@ public class MahaNamespaceExtractionConfig { @Nullable @JsonProperty(value = "kafka") - private Properties kafkaProperties; + private Properties kafkaProperties = new Properties(); - @Nullable @JsonProperty(value = "service") + @NotNull @JsonProperty(value = "lookupService") private Properties lookupServiceProperties; - @Nullable @JsonProperty(value = "rocksdb") + @NotNull @JsonProperty(value = "rocksdb") private Properties rocksDBProperties; @NotNull @JsonProperty(value = "schemaFactory") diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java index c5f0ea2c9..f8b079e38 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModule.java @@ -11,10 +11,7 @@ import com.google.inject.multibindings.MapBinder; import com.yahoo.maha.maha_druid_lookups.query.lookup.MahaLookupExtractorFactory; import com.yahoo.maha.maha_druid_lookups.query.lookup.MahaRegisteredLookupExtractionFn; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.ExtractionNamespaceCacheFactory; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.RocksDBExtractionNamespace; -import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.JDBCExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.*; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaNamespaceExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; @@ -88,6 +85,10 @@ public void configure(Binder binder) .addBinding(RocksDBExtractionNamespace.class) .to(RocksDBExtractionNamespaceCacheFactory.class) .in(LazySingleton.class); + getNamespaceFactoryMapBinder(binder) + .addBinding(MongoExtractionNamespace.class) + .to(MongoExtractionNamespaceCacheFactory.class) + .in(LazySingleton.class); LifecycleModule.register(binder, RocksDBManager.class); LifecycleModule.register(binder, KafkaManager.class); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java index 3e58e69a2..f955e492d 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBManager.java @@ -39,8 +39,8 @@ public class RocksDBManager { private static final ConcurrentMap rocksDBSnapshotMap = new ConcurrentHashMap<>(); private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String TEMPORARY_PATH = StandardSystemProperty.JAVA_IO_TMPDIR.value(); - private static final String ROCKSDB_LOCATION_PROP_NAME = "rocksdb.location"; - private static final String ROCKSDB_BLOCK_CACHE_SIZE_PROP_NAME = "rocksdb.block_cache_size"; + private static final String ROCKSDB_LOCATION_PROP_NAME = "localStorageDirectory"; + private static final String ROCKSDB_BLOCK_CACHE_SIZE_PROP_NAME = "blockCacheSize"; private static final String SNAPSHOT_FILE_NAME = "/rocksDBSnapshot"; private static final int UPLOAD_LOOKUP_AUDIT_MAX_RETRY = 3; private static final Random RANDOM = new Random(); @@ -48,7 +48,7 @@ public class RocksDBManager { private static final String STATS_KEY = "rocksdb.stats"; private static final long DEFAULT_BLOCK_CACHE_SIZE = (long)2 * 1024 * 1024 * 1024; - private String rocksdbLocation; + private String localStorageDirectory; private long blockCacheSize; private FileSystem fileSystem; @@ -63,7 +63,7 @@ public class RocksDBManager { @Inject public RocksDBManager(final MahaNamespaceExtractionConfig mahaNamespaceExtractionConfig, Configuration config) throws IOException { - this.rocksdbLocation = mahaNamespaceExtractionConfig.getRocksDBProperties().getProperty(ROCKSDB_LOCATION_PROP_NAME, TEMPORARY_PATH); + this.localStorageDirectory = mahaNamespaceExtractionConfig.getRocksDBProperties().getProperty(ROCKSDB_LOCATION_PROP_NAME, TEMPORARY_PATH); this.blockCacheSize = Long.parseLong(mahaNamespaceExtractionConfig.getRocksDBProperties().getProperty(ROCKSDB_BLOCK_CACHE_SIZE_PROP_NAME, String.valueOf(DEFAULT_BLOCK_CACHE_SIZE))); Preconditions.checkArgument(blockCacheSize > 0); this.fileSystem = FileSystem.get(config); @@ -125,13 +125,13 @@ public String createDB(final RocksDBExtractionNamespace extractionNamespace, } } - final File file = new File(String.format("%s/%s", rocksdbLocation, extractionNamespace.getNamespace())); + final File file = new File(String.format("%s/%s", localStorageDirectory, extractionNamespace.getNamespace())); if(!file.exists()) { FileUtils.forceMkdir(file); } final String localZippedFileNameWithPath = String.format("%s/%s/rocksdb_%s.zip", - rocksdbLocation, extractionNamespace.getNamespace(), loadTime); + localStorageDirectory, extractionNamespace.getNamespace(), loadTime); LOG.info(String.format("localZippedFileNameWithPath [%s]", localZippedFileNameWithPath)); final String localPath = FilenameUtils.removeExtension(localZippedFileNameWithPath); @@ -245,7 +245,7 @@ public RocksDB getDB(final String namespace) { @LifecycleStart public void start() throws IOException { RocksDB.loadLibrary(); - FileUtils.forceMkdir(new File(rocksdbLocation)); + FileUtils.forceMkdir(new File(localStorageDirectory)); } private void cleanup(String path) throws IOException { @@ -325,19 +325,19 @@ private void lookupAuditing(final String localZippedFileNameWithPath, if (dirToZip.exists() && !isSuccessMarkerPresent(successMarkerPath)) { - final File file = new File(String.format("%s/%s/%s", rocksdbLocation, "lookup_auditing", extractionNamespace.getNamespace())); + final File file = new File(String.format("%s/%s/%s", localStorageDirectory, "lookup_auditing", extractionNamespace.getNamespace())); if (!file.exists()) { FileUtils.forceMkdir(file); } final String localFileNameWithPath = String.format("%s/%s/%s/rocksdb.zip", - rocksdbLocation, "lookup_auditing", extractionNamespace.getNamespace()); + localStorageDirectory, "lookup_auditing", extractionNamespace.getNamespace()); LOG.info(String.format("localFileNameWithPath [%s]", localFileNameWithPath)); ZipUtil.pack(dirToZip, new File(localFileNameWithPath)); uploadFileForAuditing(extractionNamespace, loadTime, successMarkerPath, localFileNameWithPath); - cleanup(String.format("%s/%s/%s", rocksdbLocation, "lookup_auditing", extractionNamespace.getNamespace())); + cleanup(String.format("%s/%s/%s", localStorageDirectory, "lookup_auditing", extractionNamespace.getNamespace())); LOG.info("Uploaded lookup for auditing"); serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_UPLOAD_LOOKUP_FOR_AUDITING_SUCCESS, 1)); @@ -346,7 +346,7 @@ private void lookupAuditing(final String localZippedFileNameWithPath, } catch (Exception e) { LOG.error(e, "Caught exception while uploading lookups to HDFS for auditing"); try { - cleanup(String.format("%s/%s/%s", rocksdbLocation, "lookup_auditing", extractionNamespace.getNamespace())); + cleanup(String.format("%s/%s/%s", localStorageDirectory, "lookup_auditing", extractionNamespace.getNamespace())); if (!isSuccessMarkerPresent(successMarkerPath)) { fileSystem.delete(new Path(String.format("%s/load_time=%s/rocksdb.zip", extractionNamespace.getLookupAuditingHDFSPath(), loadTime)), false); @@ -381,7 +381,7 @@ private void uploadFileForAuditing(RocksDBExtractionNamespace extractionNamespac fileSystem.copyFromLocalFile(new Path(localFileNameWithPath), new Path(hdfsLookupAuditingPath)); final String localSuccessPath = String.format("%s/%s/%s/_SUCCESS", - rocksdbLocation, "lookup_auditing", extractionNamespace.getNamespace()); + localStorageDirectory, "lookup_auditing", extractionNamespace.getNamespace()); File successFile = new File(localSuccessPath); if (!successFile.exists()) { new FileOutputStream(successFile).close(); diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java index 7a5cc71fb..24e6a8932 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/cache/MahaNamespaceExtractionCacheManager.java @@ -81,7 +81,7 @@ public MahaNamespaceExtractionCacheManager( 15, new ThreadFactoryBuilder() .setDaemon(true) - .setNameFormat("NamespaceExtractionCacheManager-%d") + .setNameFormat("MahaNamespaceExtractionCacheManager-%d") .setPriority(Thread.MIN_PRIORITY) .build() ) @@ -214,7 +214,7 @@ public boolean scheduleAndWait( final NamespaceImplData namespaceImplData = implData.get(id); if (namespaceImplData == null) { - log.warn("NamespaceLookupExtractorFactory[%s] - deleted during start", id); + log.warn("MahaNamespaceLookupExtractorFactory[%s] - deleted during start", id); return false; } @@ -222,7 +222,7 @@ public boolean scheduleAndWait( try { success = namespaceImplData.firstRun.await(waitForFirstRun, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - log.error(e, "NamespaceLookupExtractorFactory[%s] - interrupted during start", id); + log.error(e, "MahaNamespaceLookupExtractorFactory[%s] - interrupted during start", id); } return success; } @@ -461,4 +461,7 @@ public Optional getExtractionNamespace(final String id) { return implData.containsKey(id) ? Optional.of(implData.get(id).namespace) : Optional.empty(); } + public void shutdown() { + listeningScheduledExecutorService.shutdown(); + } } diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/TestMongoServer.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/TestMongoServer.java new file mode 100644 index 000000000..486ef18b8 --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/TestMongoServer.java @@ -0,0 +1,76 @@ +package com.yahoo.maha.maha_druid_lookups; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.mongodb.BasicDBObject; +import com.mongodb.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.result.UpdateResult; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoStorageConnectorConfig; +import de.bwaldvogel.mongo.MongoServer; +import de.bwaldvogel.mongo.backend.memory.MemoryBackend; +import org.bson.Document; +import org.bson.types.ObjectId; + +import java.net.InetSocketAddress; + +public class TestMongoServer { + MongoServer mongoServer; + + protected InetSocketAddress setupMongoServer(ObjectMapper objectMapper) throws Exception { + mongoServer = new MongoServer(new MemoryBackend()); + return mongoServer.bind(); + } + + protected void createTestData(String jsonResource + , String collectionName + , ObjectMapper objectMapper + , MongoStorageConnectorConfig mongoStorageConnectorConfig) throws Exception { + MongoClient localMongoClient = mongoStorageConnectorConfig.getMongoClient(); + MongoDatabase database = localMongoClient.getDatabase(mongoStorageConnectorConfig.getDbName()); + database.createCollection(collectionName); + MongoCollection collection = database.getCollection(collectionName); + JsonNode node = objectMapper.readValue( + ClassLoader.getSystemClassLoader().getResourceAsStream(jsonResource), JsonNode.class); + for (int i = 0; i < node.size(); i++) { + JsonNode elem = node.get(i); + String json = objectMapper.writeValueAsString(elem); + Document d = Document.parse(json); + collection.insertOne(d); + } + for (Document d : collection.find()) { + System.out.println(d.toJson()); + } + localMongoClient.close(); + } + + protected void updateTestData(String jsonResource + , String collectionName + , ObjectMapper objectMapper + , MongoStorageConnectorConfig mongoStorageConnectorConfig) throws Exception { + MongoClient localMongoClient = mongoStorageConnectorConfig.getMongoClient(); + MongoDatabase database = localMongoClient.getDatabase(mongoStorageConnectorConfig.getDbName()); + MongoCollection collection = database.getCollection(collectionName); + JsonNode node = objectMapper.readValue( + ClassLoader.getSystemClassLoader().getResourceAsStream(jsonResource), JsonNode.class); + for (int i = 0; i < node.size(); i++) { + JsonNode elem = node.get(i); + String id = elem.get("_id").get("$oid").asText(); + String json = objectMapper.writeValueAsString(elem); + Document d = Document.parse(json); + UpdateResult result = collection.replaceOne(new BasicDBObject("_id", new ObjectId(id)), d); + if (result.getMatchedCount() <= 0) { + collection.insertOne(d); + } + } + for (Document d : collection.find()) { + System.out.println(d.toJson()); + } + localMongoClient.close(); + } + + protected void cleanupMongoServer() { + mongoServer.shutdownNow(); + } +} diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java new file mode 100644 index 000000000..2559a5e6f --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java @@ -0,0 +1,156 @@ +package com.yahoo.maha.maha_druid_lookups.query.lookup; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; +import com.google.common.collect.ImmutableList; +import com.google.common.io.CharStreams; +import com.google.inject.*; +import com.yahoo.maha.maha_druid_lookups.TestMongoServer; +import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoExtractionNamespace; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.MahaNamespaceExtractionModule; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; +import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaNamespaceExtractionCacheManager; +import io.druid.guice.GuiceAnnotationIntrospector; +import io.druid.guice.GuiceInjectableValues; +import io.druid.guice.GuiceInjectors; +import io.druid.guice.JsonConfigProvider; +import io.druid.guice.annotations.Self; +import io.druid.initialization.Initialization; +import io.druid.jackson.DefaultObjectMapper; +import io.druid.query.extraction.ExtractionFn; +import io.druid.query.lookup.LookupExtractor; +import io.druid.query.lookup.LookupExtractorFactory; +import io.druid.query.lookup.LookupExtractorFactoryContainer; +import io.druid.query.lookup.LookupReferencesManager; +import io.druid.server.DruidNode; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.*; + +import java.io.InputStream; +import java.io.InputStreamReader; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import java.util.Properties; + +public class MahaLookupExtractionFactoryTest extends TestMongoServer { + + Provider provider = new Provider() { + @Override + public LookupReferencesManager get() { + return getLookupReferencesManager(); + } + }; + + Injector injector = createInjector(); + + ObjectMapper objectMapper = injector.getInstance(ObjectMapper.class); + + InetSocketAddress mongoSocketAddress; + + LookupReferencesManager lookupReferencesManager; + + LookupReferencesManager getLookupReferencesManager() { + return lookupReferencesManager; + } + + private Injector createInjector() { + Injector injector = GuiceInjectors.makeStartupInjector(); + final Properties properties = injector.getInstance(Properties.class); + properties.put(MahaNamespaceExtractionModule.TYPE_PREFIX, "onHeap"); + properties.put(String.format("%s.authHeaderFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.NoopAuthHeaderFactory"); + properties.put(String.format("%s.schemaFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.TestProtobufSchemaFactory"); + properties.put(String.format("%s.lookupService.service_scheme", MahaNamespaceExtractionModule.PREFIX), "http"); + properties.put(String.format("%s.lookupService.service_port", MahaNamespaceExtractionModule.PREFIX), "4080"); + properties.put(String.format("%s.lookupService.service_nodes", MahaNamespaceExtractionModule.PREFIX), "hist1,hist2,hist3"); + properties.put(String.format("%s.rocksdb.localStorageDirectory", MahaNamespaceExtractionModule.PREFIX), "/home/y/tmp/maha-lookups"); + properties.put(String.format("%s.rocksdb.blockCacheSize", MahaNamespaceExtractionModule.PREFIX), "2147483648"); + + injector = Initialization.makeInjectorWithModules( + injector, + ImmutableList.of( + new Module() { + @Override + public void configure(Binder binder) { + JsonConfigProvider.bindInstance( + binder, + Key.get(DruidNode.class, Self.class), + new DruidNode("test-inject", null, null, null, true, false) + ); + binder.bind(LookupReferencesManager.class).toProvider(provider); + } + } + ) + ); + + return injector; + } + + @BeforeClass + public void setup() throws Exception { + mongoSocketAddress = setupMongoServer(objectMapper); + } + + @AfterClass + public void cleanup() { + cleanupMongoServer(); + } + + @Test + public void successfullyDeserializeFullNamespaceFromJSON() throws Exception { + InputStream input = Thread.currentThread().getContextClassLoader().getResourceAsStream("maha_lookup_extraction_factory.json"); + String json = String.format(CharStreams.toString(new InputStreamReader(input, StandardCharsets.UTF_8)) + , mongoSocketAddress.getHostString(), mongoSocketAddress.getPort()); + LookupExtractorFactoryContainer container = objectMapper + .readValue(json + , LookupExtractorFactoryContainer.class); + assertEquals(container.getVersion(), "v1"); + assertNotNull(container.getLookupExtractorFactory()); + LookupExtractorFactory factory = container.getLookupExtractorFactory(); + assertTrue(factory instanceof MahaLookupExtractorFactory); + MahaLookupExtractorFactory mahaFactory = (MahaLookupExtractorFactory) factory; + assertEquals(mahaFactory.getExtractionNamespace().getLookupName(), "advertiser_lookup"); + assertEquals(mahaFactory.getExtractionNamespace().getPollMs(), 30000L); + assertEquals(mahaFactory.getExtractionNamespace().getTsColumn(), "updated_at"); + assertTrue(mahaFactory.getExtractionNamespace().isCacheEnabled()); + assertTrue(mahaFactory.getExtractionNamespace() instanceof MongoExtractionNamespace); + MongoExtractionNamespace mongoExtractionNamespace = (MongoExtractionNamespace) mahaFactory.getExtractionNamespace(); + + createTestData("mongo_advertiser.json", "advertiser", objectMapper, mongoExtractionNamespace.getConnectorConfig()); + + assertTrue(factory.start()); + + MahaNamespaceExtractionCacheManager manager = injector.getInstance(MahaNamespaceExtractionCacheManager.class); + assertEquals(OnHeapMahaNamespaceExtractionCacheManager.class, manager.getClass()); + OnHeapMahaNamespaceExtractionCacheManager onHeapManager = (OnHeapMahaNamespaceExtractionCacheManager) manager; + LookupExtractor extractor = onHeapManager.getLookupExtractor("advertiser_lookup"); + assertTrue(extractor instanceof MahaLookupExtractor); + MahaLookupExtractor mahaExtractor = (MahaLookupExtractor) extractor; + assertEquals(mahaExtractor.apply("5ad10906fc7b6ecac8d41081", "name", null, null), "advertiser1"); + + lookupReferencesManager = mock(LookupReferencesManager.class); + when(lookupReferencesManager.get(any())).thenReturn(container); + + MahaRegisteredLookupExtractionFn fn = objectMapper + .readValue(Thread.currentThread() + .getContextClassLoader() + .getResourceAsStream("maha_registered_lookup_extraction_fn.json") + , MahaRegisteredLookupExtractionFn.class); + + assertEquals(fn.getLookup(), "advertiser_lookup"); + assertEquals(fn.getValueColumn(), "name"); + assertEquals(fn.getExtractionType(), ExtractionFn.ExtractionType.ONE_TO_ONE); + assertEquals(fn.getReplaceMissingValueWith(), "Unknown"); + + assertEquals(fn.apply("5ad10906fc7b6ecac8d41080"), "123"); + assertEquals(fn.apply("5ad10906fc7b6ecac8d41083"), "Unknown"); + + manager.shutdown(); + } + +} diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java index 8ff4212d6..e9fb371e5 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaRegisteredLookupExtractionFnTest.java @@ -47,7 +47,7 @@ public void testWhenCacheValueIsEmpty() { LookupReferencesManager lrm = mock(LookupReferencesManager.class); when(lrm.get(anyString())).thenReturn(lefc); - MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, true)); + MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, "advertiser_lookup", false, "", false, false, "status", null, null, true)); Assert.assertNull(fn.cache); Assert.assertEquals(fn.apply("123"), "ON"); Assert.assertEquals(fn.cache.getIfPresent("123"), "ON"); @@ -73,7 +73,7 @@ public void testWhenCacheValueIsPresent() { LookupReferencesManager lrm = mock(LookupReferencesManager.class); when(lrm.get(anyString())).thenReturn(lefc); - MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, true)); + MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, "advertiser_lookup", false, "", false, false, "status", null, null, true)); fn.ensureCache().put("123", "hola"); @@ -101,7 +101,7 @@ public void testWhenUseQueryLevelCacheIsFalse() { LookupReferencesManager lrm = mock(LookupReferencesManager.class); when(lrm.get(anyString())).thenReturn(lefc); - MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, false)); + MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, "advertiser_lookup", false, "", false, false, "status", null, null, false)); Assert.assertNull(fn.cache); Assert.assertEquals(fn.apply("123"), "ON"); @@ -128,7 +128,7 @@ public void testWhenNullValueIsPresent() { LookupReferencesManager lrm = mock(LookupReferencesManager.class); when(lrm.get(anyString())).thenReturn(lefc); - MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, objectMapper, "advertiser_lookup", false, "", false, false, "status", null, null, true)); + MahaRegisteredLookupExtractionFn fn = spy(new MahaRegisteredLookupExtractionFn(lrm, "advertiser_lookup", false, "", false, false, "status", null, null, true)); Assert.assertNull(fn.apply(null)); } diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java index 359ce47b6..9e27b579c 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespaceExtractionModuleTest.java @@ -26,7 +26,12 @@ public void testInjection() final Properties properties = injector.getInstance(Properties.class); properties.put(MahaNamespaceExtractionModule.TYPE_PREFIX, "onHeap"); properties.put(String.format("%s.authHeaderFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.NoopAuthHeaderFactory"); - properties.put(String.format("%s.schemaFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.DefaultProtobufSchemaFactory"); + properties.put(String.format("%s.schemaFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.TestProtobufSchemaFactory"); + properties.put(String.format("%s.lookupService.service_scheme", MahaNamespaceExtractionModule.PREFIX), "http"); + properties.put(String.format("%s.lookupService.service_port", MahaNamespaceExtractionModule.PREFIX), "4080"); + properties.put(String.format("%s.lookupService.service_nodes", MahaNamespaceExtractionModule.PREFIX), "hist1,hist2,hist3"); + properties.put(String.format("%s.rocksdb.localStorageDirectory", MahaNamespaceExtractionModule.PREFIX), "/home/y/tmp/maha-lookups"); + properties.put(String.format("%s.rocksdb.blockCacheSize", MahaNamespaceExtractionModule.PREFIX), "2147483648"); injector = Initialization.makeInjectorWithModules( injector, diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java index def96d8ea..41e3ffed5 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactoryTest.java @@ -12,6 +12,7 @@ import com.mongodb.client.MongoDatabase; import com.mongodb.client.model.Projections; import com.mongodb.client.result.UpdateResult; +import com.yahoo.maha.maha_druid_lookups.TestMongoServer; import com.yahoo.maha.maha_druid_lookups.query.lookup.DecodeConfig; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoExtractionNamespace; import com.yahoo.maha.maha_druid_lookups.query.lookup.namespace.MongoStorageConnectorConfig; @@ -36,7 +37,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; -public class MongoExtractionNamespaceCacheFactoryTest { +public class MongoExtractionNamespaceCacheFactoryTest extends TestMongoServer { private static final ObjectMapper objectMapper = new ObjectMapper(); MongoExtractionNamespaceCacheFactory obj = new MongoExtractionNamespaceCacheFactory(); @@ -47,18 +48,13 @@ public class MongoExtractionNamespaceCacheFactoryTest { @Mock LookupService lookupService; - MongoServer mongoServer; - MongoStorageConnectorConfig mongoStorageConnectorConfig; - MongoClient mongoClient; - private static final String DEFAULT_COLLECTION = "advertiser"; @BeforeClass public void setup() throws Exception { - mongoServer = new MongoServer(new MemoryBackend()); - InetSocketAddress serverAddress = mongoServer.bind(); + InetSocketAddress serverAddress = setupMongoServer(objectMapper); String jsonConfig = String.format("{\n" + "\t\"hosts\": \"%s:%s\",\n" + "\t\"dbName\": \"mydb\",\n" + @@ -69,55 +65,16 @@ public void setup() throws Exception { "\t}\n" + "}", serverAddress.getHostString(), serverAddress.getPort()); mongoStorageConnectorConfig = objectMapper.readValue(jsonConfig, MongoStorageConnectorConfig.class); - mongoClient = mongoStorageConnectorConfig.getMongoClient(); - createTestData(); - } - - private void createTestData() throws Exception { - MongoClient localMongoClient = mongoStorageConnectorConfig.getMongoClient(); - MongoDatabase database = localMongoClient.getDatabase(mongoStorageConnectorConfig.getDbName()); - database.createCollection(DEFAULT_COLLECTION); - MongoCollection collection = database.getCollection(DEFAULT_COLLECTION); - JsonNode node = objectMapper.readValue( - ClassLoader.getSystemClassLoader().getResourceAsStream("mongo_advertiser.json"), JsonNode.class); - for (int i = 0; i < node.size(); i++) { - JsonNode elem = node.get(i); - String json = objectMapper.writeValueAsString(elem); - Document d = Document.parse(json); - collection.insertOne(d); - } - for (Document d : collection.find()) { - System.out.println(d.toJson()); - } - localMongoClient.close(); + createTestData("mongo_advertiser.json", DEFAULT_COLLECTION, objectMapper, mongoStorageConnectorConfig); } private void updateTestData(String jsonResource) throws Exception { - MongoClient localMongoClient = mongoStorageConnectorConfig.getMongoClient(); - MongoDatabase database = localMongoClient.getDatabase(mongoStorageConnectorConfig.getDbName()); - MongoCollection collection = database.getCollection(DEFAULT_COLLECTION); - JsonNode node = objectMapper.readValue( - ClassLoader.getSystemClassLoader().getResourceAsStream(jsonResource), JsonNode.class); - for (int i = 0; i < node.size(); i++) { - JsonNode elem = node.get(i); - String id = elem.get("_id").get("$oid").asText(); - String json = objectMapper.writeValueAsString(elem); - Document d = Document.parse(json); - UpdateResult result = collection.replaceOne(new BasicDBObject("_id", new ObjectId(id)), d); - if (result.getMatchedCount() <= 0) { - collection.insertOne(d); - } - } - for (Document d : collection.find()) { - System.out.println(d.toJson()); - } - localMongoClient.close(); + super.updateTestData(jsonResource, DEFAULT_COLLECTION, objectMapper, mongoStorageConnectorConfig); } @AfterClass - public void cleanup() throws Exception { - mongoClient.close(); - mongoServer.shutdownNow(); + public void cleanup() { + cleanupMongoServer(); } @BeforeTest diff --git a/druid-lookups/src/test/resources/maha_lookup_extraction_factory.json b/druid-lookups/src/test/resources/maha_lookup_extraction_factory.json new file mode 100644 index 000000000..10ecd9af5 --- /dev/null +++ b/druid-lookups/src/test/resources/maha_lookup_extraction_factory.json @@ -0,0 +1,34 @@ +{ + "version": "v1", + "lookupExtractorFactory": { + "type": "cachedNamespace", + "firstCacheTimeout": 30000, + "injective": true, + "extractionNamespace" : { + "type": "mahamongo", + "connectorConfig": { + "hosts": "%s:%s", + "dbName": "advertiser", + "clientOptions": { + "socketTimeout": 30000 + } + }, + "collectionName": "advertiser", + "tsColumn": "updated_at", + "tsColumnEpochInteger": true, + "pollPeriod": "PT30S", + "cacheEnabled": true, + "lookupName": "advertiser_lookup", + "documentProcessor": { + "type": "flatmultivalue", + "columnList": [ + "name", + "currency", + "status" + ], + "primaryKeyColumn": "_id" + }, + "mongoClientRetryCount": 3 + } + } +} \ No newline at end of file diff --git a/druid-lookups/src/test/resources/maha_registered_lookup_extraction_fn.json b/druid-lookups/src/test/resources/maha_registered_lookup_extraction_fn.json new file mode 100644 index 000000000..c807e7fa3 --- /dev/null +++ b/druid-lookups/src/test/resources/maha_registered_lookup_extraction_fn.json @@ -0,0 +1,7 @@ +{ + "type" : "mahaRegisteredLookup", + "lookup" : "advertiser_lookup", + "replaceMissingValueWith": "Unknown", + "injective": true, + "valueColumn": "name" +} \ No newline at end of file From a1cb04864bfbd94ad451dccce776fbb340235b2e Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 16:32:32 -0800 Subject: [PATCH 04/15] Cleanup imports --- .../lookup/MahaLookupExtractionFactoryTest.java | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java index 2559a5e6f..e08947925 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java @@ -1,7 +1,6 @@ package com.yahoo.maha.maha_druid_lookups.query.lookup; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair; import com.google.common.collect.ImmutableList; import com.google.common.io.CharStreams; import com.google.inject.*; @@ -10,13 +9,10 @@ import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.MahaNamespaceExtractionModule; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.MahaNamespaceExtractionCacheManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.cache.OnHeapMahaNamespaceExtractionCacheManager; -import io.druid.guice.GuiceAnnotationIntrospector; -import io.druid.guice.GuiceInjectableValues; import io.druid.guice.GuiceInjectors; import io.druid.guice.JsonConfigProvider; import io.druid.guice.annotations.Self; import io.druid.initialization.Initialization; -import io.druid.jackson.DefaultObjectMapper; import io.druid.query.extraction.ExtractionFn; import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractorFactory; @@ -27,17 +23,17 @@ import org.testng.annotations.BeforeClass; import org.testng.annotations.Test; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.testng.Assert.*; - import java.io.InputStream; import java.io.InputStreamReader; import java.net.InetSocketAddress; import java.nio.charset.StandardCharsets; import java.util.Properties; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.*; + public class MahaLookupExtractionFactoryTest extends TestMongoServer { Provider provider = new Provider() { From 60b6c4a196f4497f586df58139e780c410dd9a01 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 17:42:33 -0800 Subject: [PATCH 05/15] Fix druid query generator --- .../maha/core/query/druid/DruidQueryGenerator.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala b/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala index 8b4c6a343..f63fcabd5 100644 --- a/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala +++ b/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala @@ -1217,17 +1217,17 @@ class DruidQueryGenerator(queryOptimizer: DruidQueryOptimizer renderColumnWithAlias(fact, column, alias) case lookupFunc@LOOKUP(lookupNamespace, valueColumn, dimensionOverrideMap) => - val regExFn = new MahaRegisteredLookupExtractionFn(null, null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) (new ExtractionDimensionSpec(primaryColumn.alias.getOrElse(primaryColumn.name), alias, getDimValueType(column), regExFn, null), Option.empty) case lookupFunc@LOOKUP_WITH_EMPTY_VALUE_OVERRIDE(lookupNamespace, valueColumn, overrideValue, dimensionOverrideMap) => - val regExFn = new MahaRegisteredLookupExtractionFn(null, null, lookupNamespace, false, overrideValue, false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, overrideValue, false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) (new ExtractionDimensionSpec(primaryColumn.alias.getOrElse(primaryColumn.name), alias, getDimValueType(column), regExFn, null), Option.empty) case lookupFunc@LOOKUP_WITH_DECODE(lookupNamespace, valueColumn, dimensionOverrideMap, args@_*) => - val regExFn = new MahaRegisteredLookupExtractionFn(null, null, lookupNamespace, false, lookupFunc.default.getOrElse(DruidQuery.replaceMissingValueWith), false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, lookupFunc.default.getOrElse(DruidQuery.replaceMissingValueWith), false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) val mapLookup = new MapLookupExtractor(lookupFunc.map.asJava, false) val mapExFn = new LookupExtractionFn(mapLookup, false, lookupFunc.default.getOrElse(null), false, true) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) @@ -1235,7 +1235,7 @@ class DruidQueryGenerator(queryOptimizer: DruidQueryOptimizer Option.apply(new ExtractionDimensionSpec(alias, alias, getDimValueType(column), mapExFn, null))) case lookupFunc@LOOKUP_WITH_DECODE_RETAIN_MISSING_VALUE(lookupNamespace, valueColumn, retainMissingValue, injective, dimensionOverrideMap, args@_*) => - val regExFn = new MahaRegisteredLookupExtractionFn(null, null, lookupNamespace, false, lookupFunc.lookupWithDecode.default.getOrElse(DruidQuery.replaceMissingValueWith), false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, lookupFunc.lookupWithDecode.default.getOrElse(DruidQuery.replaceMissingValueWith), false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) val mapLookup = new MapLookupExtractor(lookupFunc.lookupWithDecode.map.asJava, false) val mapExFn = new LookupExtractionFn(mapLookup, retainMissingValue, null, injective, true) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) @@ -1248,12 +1248,12 @@ class DruidQueryGenerator(queryOptimizer: DruidQueryOptimizer decodeConfig.setValueToCheck(valueToCheck) decodeConfig.setColumnIfValueMatched(columnIfValueMatched) decodeConfig.setColumnIfValueNotMatched(columnIfValueNotMatched) - val regExFn = new MahaRegisteredLookupExtractionFn(null, null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, null, decodeConfig, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, null, decodeConfig, dimensionOverrideMap.asJava, useQueryLevelCache) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) (new ExtractionDimensionSpec(primaryColumn.alias.getOrElse(primaryColumn.name), alias, getDimValueType(column), regExFn, null), Option.empty) case lookupFunc@LOOKUP_WITH_TIMEFORMATTER(lookupNamespace, valueColumn, inputFormat, resultFormat, dimensionOverrideMap) => - val regExFn = new MahaRegisteredLookupExtractionFn(null, null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, valueColumn, null, dimensionOverrideMap.asJava, useQueryLevelCache) val timeFormatFn = new TimeDimExtractionFn(inputFormat, resultFormat) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) (new ExtractionDimensionSpec(primaryColumn.alias.getOrElse(primaryColumn.name), alias, getDimValueType(column), regExFn, null), From 3ff2b8499b2c7ac12f51b06c148055d0d14ae887 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 17:56:02 -0800 Subject: [PATCH 06/15] Cleanup pom --- druid-lookups/pom.xml | 65 +++++++------------ .../MongoExtractionNamespaceCacheFactory.java | 2 +- 2 files changed, 24 insertions(+), 43 deletions(-) diff --git a/druid-lookups/pom.xml b/druid-lookups/pom.xml index 1cc6d7ce2..92aa64b5f 100644 --- a/druid-lookups/pom.xml +++ b/druid-lookups/pom.xml @@ -24,7 +24,7 @@ org.apache.hadoop hadoop-client 2.3.0 - compile + provided commons-cli @@ -116,21 +116,25 @@ io.druid druid-api ${druid.version} + provided io.druid druid-processing ${druid.version} + provided io.druid druid-server ${druid.version} + provided io.druid druid-common ${druid.version} + provided com.google.guava @@ -142,49 +146,31 @@ com.google.guava guava 16.0.1 + provided com.google.protobuf protobuf-java ${protobuf.version} + provided - org.apache.kafka - kafka_2.11 - 0.10.2.0 - - - log4j - log4j - - - org.apache.zookeeper - zookeeper - - - org.slf4j - slf4j-api - - - net.jpountz.lz4 - lz4 - - - org.slf4j - slf4j-log4j12 - - + com.googlecode.json-simple + json-simple + 1.1.1 + provided + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + provided org.apache.kafka kafka-clients 0.10.2.0 - - com.googlecode.json-simple - json-simple - 1.1.1 - org.rocksdb rocksdbjni @@ -201,6 +187,11 @@ mongo-java-driver ${mongo-java-driver.version} + + com.github.ben-manes.caffeine + caffeine + ${caffeine.version} + de.bwaldvogel @@ -249,6 +240,7 @@ org.testng testng 6.8.7 + test org.mockito @@ -256,17 +248,6 @@ ${mockito.version} test - - com.fasterxml.jackson.core - jackson-annotations - ${jackson.version} - compile - - - com.github.ben-manes.caffeine - caffeine - ${caffeine.version} - diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java index 93c78be44..22c6ecccb 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java @@ -2,6 +2,7 @@ // Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace; +import com.google.common.collect.Lists; import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; @@ -26,7 +27,6 @@ import org.bson.Document; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import org.testng.collections.Lists; /** * From c39d21dd9ddd526c2a4671a54e3f5d4c388af9d7 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 18:14:58 -0800 Subject: [PATCH 07/15] Add assembly of zip --- druid-lookups/pom.xml | 22 +++++++++++++++++++ .../src/main/assembly/druid-lookups.xml | 20 +++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 druid-lookups/src/main/assembly/druid-lookups.xml diff --git a/druid-lookups/pom.xml b/druid-lookups/pom.xml index 92aa64b5f..cf599c094 100644 --- a/druid-lookups/pom.xml +++ b/druid-lookups/pom.xml @@ -256,4 +256,26 @@ https://api.bintray.com/maven/yahoo/maven/maha-druid-lookups/;publish=1 + + + + org.apache.maven.plugins + maven-assembly-plugin + + + package + + single + + + false + + src/main/assembly/druid-lookups.xml + + + + + + + diff --git a/druid-lookups/src/main/assembly/druid-lookups.xml b/druid-lookups/src/main/assembly/druid-lookups.xml new file mode 100644 index 000000000..86905fda7 --- /dev/null +++ b/druid-lookups/src/main/assembly/druid-lookups.xml @@ -0,0 +1,20 @@ + + zip + true + + + zip + + + + / + + org.slf4j:slf4j-api + net.jpountz.lz4:lz4 + org.xerial.snappy:snappy-java + + + + From 03c701dc5301b50a074250011a59089a2d2c117e Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 18:24:23 -0800 Subject: [PATCH 08/15] Add jacoco for coverage --- druid-lookups/pom.xml | 98 +++++++++++++++++++++++++++++++++++++++---- 1 file changed, 91 insertions(+), 7 deletions(-) diff --git a/druid-lookups/pom.xml b/druid-lookups/pom.xml index cf599c094..22b011f7f 100644 --- a/druid-lookups/pom.xml +++ b/druid-lookups/pom.xml @@ -1,7 +1,8 @@ - + maha-parent com.yahoo.maha @@ -250,14 +251,48 @@ - - - bintray-maha-druid-lookups-repo - https://api.bintray.com/maven/yahoo/maven/maha-druid-lookups/;publish=1 - - + + + bintray-maha-druid-lookups-repo + https://api.bintray.com/maven/yahoo/maven/maha-druid-lookups/;publish=1 + + + + org.apache.maven.plugins + maven-compiler-plugin + ${maven.compiler.plugin.version} + + 1.8 + 1.8 + + + + org.apache.maven.plugins + maven-surefire-plugin + ${maven-surefire-plugin.version} + + 7 + true + -Xmx2048m -Djava.library.path=/home/y/lib64 ${surefireArgLine} + target/FORK_DIRECTORY_${surefire.forkNumber} + true + + + + org.scoverage + scoverage-maven-plugin + ${scoverage.plugin.version} + + + none + + + + true + + org.apache.maven.plugins maven-assembly-plugin @@ -276,6 +311,55 @@ + + org.jacoco + jacoco-maven-plugin + 0.8.1 + + + prepare-agent + + prepare-agent + + + surefireArgLine + + + + report + test + + report + + + + check + test + + check + + + + + + BUNDLE + + + + COMPLEXITY + 0.30 + + + COVEREDRATIO + 0.38 + + + + + + + + From 573e133a911d96564582ebde1cb28342d58af6b2 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 22 Feb 2019 18:40:28 -0800 Subject: [PATCH 09/15] Fix requirement of valueColumn --- .../com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala | 2 +- .../yahoo/maha/core/query/druid/DruidQueryGeneratorTest.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala b/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala index f63fcabd5..9b9a818e1 100644 --- a/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala +++ b/core/src/main/scala/com/yahoo/maha/core/query/druid/DruidQueryGenerator.scala @@ -1248,7 +1248,7 @@ class DruidQueryGenerator(queryOptimizer: DruidQueryOptimizer decodeConfig.setValueToCheck(valueToCheck) decodeConfig.setColumnIfValueMatched(columnIfValueMatched) decodeConfig.setColumnIfValueNotMatched(columnIfValueNotMatched) - val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, null, decodeConfig, dimensionOverrideMap.asJava, useQueryLevelCache) + val regExFn = new MahaRegisteredLookupExtractionFn(null, lookupNamespace, false, DruidQuery.replaceMissingValueWith, false, true, columnToCheck, decodeConfig, dimensionOverrideMap.asJava, useQueryLevelCache) val primaryColumn = queryContext.factBestCandidate.fact.publicDimToForeignKeyColMap(db.publicDim.name) (new ExtractionDimensionSpec(primaryColumn.alias.getOrElse(primaryColumn.name), alias, getDimValueType(column), regExFn, null), Option.empty) diff --git a/core/src/test/scala/com/yahoo/maha/core/query/druid/DruidQueryGeneratorTest.scala b/core/src/test/scala/com/yahoo/maha/core/query/druid/DruidQueryGeneratorTest.scala index 980493d2a..1ae09f04f 100644 --- a/core/src/test/scala/com/yahoo/maha/core/query/druid/DruidQueryGeneratorTest.scala +++ b/core/src/test/scala/com/yahoo/maha/core/query/druid/DruidQueryGeneratorTest.scala @@ -1532,7 +1532,7 @@ class DruidQueryGeneratorTest extends BaseDruidQueryGeneratorTest { assert(queryPipelineTry.isSuccess, queryPipelineTry.errorMessage("Fail to get the query pipeline")) val result = queryPipelineTry.toOption.get.queryChain.drivingQuery.asInstanceOf[DruidQuery[_]].asString - val json = """\{"queryType":"groupBy","dataSource":\{"type":"query","query":\{"queryType":"groupBy","dataSource":\{"type":"table","name":"fact1"\},"intervals":\{"type":"intervals","intervals":\[".*"\]\},"virtualColumns":\[\],"filter":\{"type":"and","fields":\[\{"type":"or","fields":\[\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\}\]\},\{"type":"selector","dimension":"advertiser_id","value":"12345"\}\]\},"granularity":\{"type":"all"\},"dimensions":\[\{"type":"default","dimension":"statsDate","outputName":"Day","outputType":"STRING"\},\{"type":"extraction","dimension":"advertiser_id","outputName":"Advertiser Status","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"advertiser_lookup","retainMissingValue":false,"replaceMissingValueWith":"MAHA_LOOKUP_EMPTY","injective":false,"optimize":true,"valueColumn":"status","dimensionOverrideMap":\{\},"useQueryLevelCache":true\}\},\{"type":"extraction","dimension":"advertiser_id","outputName":"Currency","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"advertiser_lookup","retainMissingValue":false,"replaceMissingValueWith":"MAHA_LOOKUP_EMPTY","injective":false,"optimize":true,"valueColumn":"currency","dimensionOverrideMap":\{"-3":"Unknown","":"Unknown"\},"useQueryLevelCache":true\}\},\{"type":"extraction","dimension":"advertiser_id","outputName":"Timezone","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"advertiser_lookup","retainMissingValue":false,"replaceMissingValueWith":"MAHA_LOOKUP_EMPTY","injective":false,"optimize":true,"decode":\{"columnToCheck":"timezone","valueToCheck":"US","columnIfValueMatched":"timezone","columnIfValueNotMatched":"currency"\},"dimensionOverrideMap":\{\},"useQueryLevelCache":true\}\},\{"type":"extraction","dimension":"external_id","outputName":"External Site Name","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"site_lookup","retainMissingValue":false,"replaceMissingValueWith":"Others","injective":false,"optimize":true,"valueColumn":"external_site_name","dimensionOverrideMap":\{\},"useQueryLevelCache":false\}\}\],"aggregations":\[\{"type":"longSum","name":"Impressions","fieldName":"impressions"\},\{"type":"roundingDoubleSum","name":"_sum_avg_bid","fieldName":"avg_bid","scale":10,"enableRoundingDoubleSumAggregatorFactory":true\},\{"type":"count","name":"_count_avg_bid"\}\],"postAggregations":\[\{"type":"arithmetic","name":"Average Bid","fn":"/","fields":\[\{"type":"fieldAccess","name":"_sum_avg_bid","fieldName":"_sum_avg_bid"\},\{"type":"fieldAccess","name":"_count_avg_bid","fieldName":"_count_avg_bid"\}\]\}\],"limitSpec":\{"type":"default","columns":\[\],"limit":220\},"context":\{"groupByStrategy":"v2","applyLimitPushDown":"false","uncoveredIntervalsLimit":1,"groupByIsSingleThreaded":true,"timeout":5000,"queryId":".*"\},"descending":false\}\},"intervals":\{"type":"intervals","intervals":\[".*"\]\},"virtualColumns":\[\],"filter":\{"type":"and","fields":\[\{"type":"selector","dimension":"Advertiser Status","value":"ON"\},\{"type":"selector","dimension":"Currency","value":"USD"\}\]\},"granularity":\{"type":"all"\},"dimensions":\[\{"type":"default","dimension":"Day","outputName":"Day","outputType":"STRING"\},\{"type":"default","dimension":"Advertiser Status","outputName":"Advertiser Status","outputType":"STRING"\},\{"type":"default","dimension":"Currency","outputName":"Currency","outputType":"STRING"\},\{"type":"default","dimension":"Timezone","outputName":"Timezone","outputType":"STRING"\},\{"type":"extraction","dimension":"External Site Name","outputName":"External Site Name","outputType":"STRING","extractionFn":\{"type":"lookup","lookup":\{"type":"map","map":\{"null":"Others","":"Others"\},"isOneToOne":false\},"retainMissingValue":true,"injective":true,"optimize":true\}\}\],"aggregations":\[\{"type":"longSum","name":"Impressions","fieldName":"Impressions"\},\{"type":"roundingDoubleSum","name":"_sum_avg_bid","fieldName":"_sum_avg_bid","scale":10,"enableRoundingDoubleSumAggregatorFactory":true\},\{"type":"count","name":"_count_avg_bid"\}\],"postAggregations":\[\{"type":"arithmetic","name":"Average Bid","fn":"/","fields":\[\{"type":"fieldAccess","name":"_sum_avg_bid","fieldName":"_sum_avg_bid"\},\{"type":"fieldAccess","name":"_count_avg_bid","fieldName":"_count_avg_bid"\}\]\}\],"limitSpec":\{"type":"default","columns":\[\],"limit":220\},"context":\{"groupByStrategy":"v2","applyLimitPushDown":"false","uncoveredIntervalsLimit":1,"groupByIsSingleThreaded":true,"timeout":5000,"queryId":".*"\},"descending":false\}""" + val json = """\{"queryType":"groupBy","dataSource":\{"type":"query","query":\{"queryType":"groupBy","dataSource":\{"type":"table","name":"fact1"\},"intervals":\{"type":"intervals","intervals":\[".*"\]\},"virtualColumns":\[\],"filter":\{"type":"and","fields":\[\{"type":"or","fields":\[\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\},\{"type":"selector","dimension":"statsDate","value":".*"\}\]\},\{"type":"selector","dimension":"advertiser_id","value":"12345"\}\]\},"granularity":\{"type":"all"\},"dimensions":\[\{"type":"default","dimension":"statsDate","outputName":"Day","outputType":"STRING"\},\{"type":"extraction","dimension":"advertiser_id","outputName":"Advertiser Status","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"advertiser_lookup","retainMissingValue":false,"replaceMissingValueWith":"MAHA_LOOKUP_EMPTY","injective":false,"optimize":true,"valueColumn":"status","dimensionOverrideMap":\{\},"useQueryLevelCache":true\}\},\{"type":"extraction","dimension":"advertiser_id","outputName":"Currency","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"advertiser_lookup","retainMissingValue":false,"replaceMissingValueWith":"MAHA_LOOKUP_EMPTY","injective":false,"optimize":true,"valueColumn":"currency","dimensionOverrideMap":\{"-3":"Unknown","":"Unknown"\},"useQueryLevelCache":true\}\},\{"type":"extraction","dimension":"advertiser_id","outputName":"Timezone","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"advertiser_lookup","retainMissingValue":false,"replaceMissingValueWith":"MAHA_LOOKUP_EMPTY","injective":false,"optimize":true,"valueColumn":"timezone","decode":\{"columnToCheck":"timezone","valueToCheck":"US","columnIfValueMatched":"timezone","columnIfValueNotMatched":"currency"\},"dimensionOverrideMap":\{\},"useQueryLevelCache":true\}\},\{"type":"extraction","dimension":"external_id","outputName":"External Site Name","outputType":"STRING","extractionFn":\{"type":"mahaRegisteredLookup","lookup":"site_lookup","retainMissingValue":false,"replaceMissingValueWith":"Others","injective":false,"optimize":true,"valueColumn":"external_site_name","dimensionOverrideMap":\{\},"useQueryLevelCache":false\}\}\],"aggregations":\[\{"type":"longSum","name":"Impressions","fieldName":"impressions"\},\{"type":"roundingDoubleSum","name":"_sum_avg_bid","fieldName":"avg_bid","scale":10,"enableRoundingDoubleSumAggregatorFactory":true\},\{"type":"count","name":"_count_avg_bid"\}\],"postAggregations":\[\{"type":"arithmetic","name":"Average Bid","fn":"/","fields":\[\{"type":"fieldAccess","name":"_sum_avg_bid","fieldName":"_sum_avg_bid"\},\{"type":"fieldAccess","name":"_count_avg_bid","fieldName":"_count_avg_bid"\}\]\}\],"limitSpec":\{"type":"default","columns":\[\],"limit":220\},"context":\{"groupByStrategy":"v2","applyLimitPushDown":"false","uncoveredIntervalsLimit":1,"groupByIsSingleThreaded":true,"timeout":5000,"queryId":".*"\},"descending":false\}\},"intervals":\{"type":"intervals","intervals":\[".*"\]\},"virtualColumns":\[\],"filter":\{"type":"and","fields":\[\{"type":"selector","dimension":"Advertiser Status","value":"ON"\},\{"type":"selector","dimension":"Currency","value":"USD"\}\]\},"granularity":\{"type":"all"\},"dimensions":\[\{"type":"default","dimension":"Day","outputName":"Day","outputType":"STRING"\},\{"type":"default","dimension":"Advertiser Status","outputName":"Advertiser Status","outputType":"STRING"\},\{"type":"default","dimension":"Currency","outputName":"Currency","outputType":"STRING"\},\{"type":"default","dimension":"Timezone","outputName":"Timezone","outputType":"STRING"\},\{"type":"extraction","dimension":"External Site Name","outputName":"External Site Name","outputType":"STRING","extractionFn":\{"type":"lookup","lookup":\{"type":"map","map":\{"null":"Others","":"Others"\},"isOneToOne":false\},"retainMissingValue":true,"injective":true,"optimize":true\}\}\],"aggregations":\[\{"type":"longSum","name":"Impressions","fieldName":"Impressions"\},\{"type":"roundingDoubleSum","name":"_sum_avg_bid","fieldName":"_sum_avg_bid","scale":10,"enableRoundingDoubleSumAggregatorFactory":true\},\{"type":"count","name":"_count_avg_bid"\}\],"postAggregations":\[\{"type":"arithmetic","name":"Average Bid","fn":"/","fields":\[\{"type":"fieldAccess","name":"_sum_avg_bid","fieldName":"_sum_avg_bid"\},\{"type":"fieldAccess","name":"_count_avg_bid","fieldName":"_count_avg_bid"\}\]\}\],"limitSpec":\{"type":"default","columns":\[\],"limit":220\},"context":\{"groupByStrategy":"v2","applyLimitPushDown":"false","uncoveredIntervalsLimit":1,"groupByIsSingleThreaded":true,"timeout":5000,"queryId":".*"\},"descending":false\}""" result should fullyMatch regex json } From cc8ffff6ce691da8cb1caecd00b9a9a1991fb8f1 Mon Sep 17 00:00:00 2001 From: patelh Date: Sat, 23 Feb 2019 11:28:20 -0800 Subject: [PATCH 10/15] Add noop schema factory --- .../DefaultProtobufSchemaFactoryConfig.java | 9 ------- .../entity/NoopProtobufSchemaFactory.java | 24 +++++++++++++++++++ .../MahaLookupExtractionFactoryTest.java | 2 +- 3 files changed, 25 insertions(+), 10 deletions(-) delete mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/DefaultProtobufSchemaFactoryConfig.java create mode 100644 druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/NoopProtobufSchemaFactory.java diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/DefaultProtobufSchemaFactoryConfig.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/DefaultProtobufSchemaFactoryConfig.java deleted file mode 100644 index b7d064655..000000000 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/DefaultProtobufSchemaFactoryConfig.java +++ /dev/null @@ -1,9 +0,0 @@ -// Copyright 2018, Yahoo Holdings Inc. -// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. -package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity; - -/** - * Created by hiral on 9/6/18. - */ -public class DefaultProtobufSchemaFactoryConfig { -} diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/NoopProtobufSchemaFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/NoopProtobufSchemaFactory.java new file mode 100644 index 000000000..eaa06f833 --- /dev/null +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/entity/NoopProtobufSchemaFactory.java @@ -0,0 +1,24 @@ +// Copyright 2017, Yahoo Holdings Inc. +// Licensed under the terms of the Apache License 2.0. Please see LICENSE file in project root for terms. +package com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.Parser; + +public class NoopProtobufSchemaFactory implements ProtobufSchemaFactory { + + public Descriptors.Descriptor getProtobufDescriptor(final String messageType) { + throw new UnsupportedOperationException("please configure appropriate factory!"); + } + + public Parser getProtobufParser(final String messageType) { + throw new UnsupportedOperationException("please configure appropriate factory!"); + + } + + public Message.Builder getProtobufMessageBuilder(final String messageType) { + throw new UnsupportedOperationException("please configure appropriate factory!"); + } + +} diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java index e08947925..8e754ce65 100644 --- a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/MahaLookupExtractionFactoryTest.java @@ -60,7 +60,7 @@ private Injector createInjector() { final Properties properties = injector.getInstance(Properties.class); properties.put(MahaNamespaceExtractionModule.TYPE_PREFIX, "onHeap"); properties.put(String.format("%s.authHeaderFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.NoopAuthHeaderFactory"); - properties.put(String.format("%s.schemaFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.TestProtobufSchemaFactory"); + properties.put(String.format("%s.schemaFactory", MahaNamespaceExtractionModule.PREFIX), "com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.NoopProtobufSchemaFactory"); properties.put(String.format("%s.lookupService.service_scheme", MahaNamespaceExtractionModule.PREFIX), "http"); properties.put(String.format("%s.lookupService.service_port", MahaNamespaceExtractionModule.PREFIX), "4080"); properties.put(String.format("%s.lookupService.service_nodes", MahaNamespaceExtractionModule.PREFIX), "hist1,hist2,hist3"); From 73e6c6afade0738d6bda803bf58bdb8a60830b4b Mon Sep 17 00:00:00 2001 From: patelh Date: Mon, 25 Feb 2019 12:34:09 -0800 Subject: [PATCH 11/15] Emit mongo lookup metrics --- .../query/lookup/RocksDBLookupExtractor.java | 3 +-- .../MahaNamespacesCacheResource.java | 2 +- .../MongoExtractionNamespaceCacheFactory.java | 24 ++++++++++++++----- .../lookup/namespace/MonitoringConstants.java | 13 ++++++---- ...ocksDBExtractionNamespaceCacheFactory.java | 4 ++-- 5 files changed, 31 insertions(+), 15 deletions(-) diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java index 590b06b2a..35e98fe9e 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/RocksDBLookupExtractor.java @@ -25,7 +25,6 @@ import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.MonitoringConstants; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.RocksDBManager; import com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.ProtobufSchemaFactory; -import io.druid.query.lookup.LookupExtractor; import org.rocksdb.RocksDB; import javax.annotation.Nullable; @@ -113,7 +112,7 @@ public String apply(@NotNull String key, @NotNull String valueColumn, DecodeConf extractionNamespace.getMissingLookupConfig().getMissingLookupKafkaTopic(), key); missingLookupCache.put(key, extractionNamespaceAsByteArray); - serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_PUBLISH_MISSING_LOOKUP_SUCESS, 1)); + serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_PUBLISH_MISSING_LOOKUP_SUCCESS, 1)); } return null; } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java index c4dea1559..07d1c6719 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MahaNamespacesCacheResource.java @@ -106,7 +106,7 @@ public Response getCacheValue(@PathParam("namespace") String namespace, .getCacheSize(extractionNamespace.get(), mahaNamespaceExtractionCacheManager.getCacheMap(namespace)).getBytes(); } - serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_GET_CACHE_VALUE_SUCESS, 1)); + serviceEmitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_GET_CACHE_VALUE_SUCCESS, 1)); return Response.ok().entity(response).build(); } } catch (Exception ex) { diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java index 22c6ecccb..ac9e7fe48 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MongoExtractionNamespaceCacheFactory.java @@ -6,6 +6,7 @@ import com.google.inject.Inject; import com.metamx.common.logger.Logger; import com.metamx.emitter.service.ServiceEmitter; +import com.metamx.emitter.service.ServiceMetricEvent; import com.mongodb.MongoClient; import com.mongodb.client.FindIterable; import com.mongodb.client.MongoCollection; @@ -37,12 +38,7 @@ public class MongoExtractionNamespaceCacheFactory DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.SSS"); private static final int[] BACKOFF_MILLIS = new int[]{100, 200, 400, 800, 1600, 3200, 6400, 12800}; private static final Logger LOG = new Logger(MongoExtractionNamespaceCacheFactory.class); - private static final String COMMA_SEPARATOR = ","; private static final String ID_FIELD = "_id"; - //private static final String FIRST_TIME_CACHING_WHERE_CLAUSE = " WHERE LAST_UPDATED <= :lastUpdatedTimeStamp"; - //private static final String SUBSEQUENT_CACHING_WHERE_CLAUSE = " WHERE LAST_UPDATED > :lastUpdatedTimeStamp"; - //private static final int FETCH_SIZE = 10000; - //private final ConcurrentMap dbiCache = new ConcurrentHashMap<>(); private final ConcurrentMap mongoClientCache = new ConcurrentHashMap<>(); @Inject LookupService lookupService; @@ -69,6 +65,7 @@ public String call() throws Exception { return new Callable() { @Override public String call() { + long startMillis = System.currentTimeMillis(); if (LOG.isDebugEnabled()) { LOG.debug("Updating [%s]", id); } @@ -105,7 +102,10 @@ public String call() { break; } catch (Exception e) { - LOG.error(e, "Failed to create mongo client, numAttempt=%s hosts=%s database=%s" + emitter.emit(ServiceMetricEvent.builder() + .setDimension(MonitoringConstants.MAHA_LOOKUP_NAME, extractionNamespace.getLookupName()) + .build(MonitoringConstants.MAHA_LOOKUP_MONGO_DATABASE_OR_COLLECTION_FAILURE, 1)); + LOG.error(e, "Failed to get database or collection, numAttempt=%s hosts=%s database=%s" , numAttempts , extractionNamespace.getConnectorConfig().getHosts() , extractionNamespace.getConnectorConfig().getDbName() @@ -145,6 +145,9 @@ public String call() { maxTime = docTime; } } catch (Exception e) { + emitter.emit(ServiceMetricEvent.builder() + .setDimension(MonitoringConstants.MAHA_LOOKUP_NAME, extractionNamespace.getLookupName()) + .build(MonitoringConstants.MAHA_LOOKUP_MONGO_DOCUMENT_PROCESS_FAILURE, 1)); LOG.error(e, "collectionName=%s tsColumn=%s failed to process document document=%s" , extractionNamespace.getCollectionName() , extractionNamespace.getTsColumn() @@ -153,6 +156,9 @@ public String call() { } } catch (Exception e) { + emitter.emit(ServiceMetricEvent.builder() + .setDimension(MonitoringConstants.MAHA_LOOKUP_NAME, extractionNamespace.getLookupName()) + .build(MonitoringConstants.MAHA_LOOKUP_MONGO_DOCUMENT_PROCESS_FAILURE, 1)); LOG.error(e, "collectionName=%s tsColumn=%s failed to process document document=%s" , extractionNamespace.getCollectionName() , extractionNamespace.getTsColumn() @@ -177,6 +183,9 @@ public String call() { } + emitter.emit(ServiceMetricEvent.builder() + .setDimension(MonitoringConstants.MAHA_LOOKUP_NAME, extractionNamespace.getLookupName()) + .build(MonitoringConstants.MAHA_LOOKUP_MONGO_PROCESSING_TIME, System.currentTimeMillis() - startMillis)); LOG.info("Finished loading %d values for extractionNamespace[%s]", cache.size(), id); return String.format("%d", extractionNamespace.getPreviousLastUpdateTime()); } @@ -208,6 +217,9 @@ private MongoClient ensureMongoClient(String id, MongoExtractionNamespace namesp } break; } catch (Exception e) { + emitter.emit(ServiceMetricEvent.builder() + .setDimension(MonitoringConstants.MAHA_LOOKUP_NAME, namespace.getLookupName()) + .build(MonitoringConstants.MAHA_LOOKUP_MONGO_CLIENT_FAILURE, 1)); LOG.error(e, "Failed to create mongo client, numAttempt=%s hosts=%s database=%s" , numAttempts , namespace.getConnectorConfig().getHosts() diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MonitoringConstants.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MonitoringConstants.java index 5e72a00c9..59d839531 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MonitoringConstants.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/MonitoringConstants.java @@ -4,16 +4,21 @@ public class MonitoringConstants { - public static final String MAHA_LOOKUP_ROCKSDB_OPEN_SUCESS = "maha_lookup/rocksdb_open_success"; + public static final String MAHA_LOOKUP_ROCKSDB_OPEN_SUCCESS = "maha_lookup/rocksdb_open_success"; public static final String MAHA_LOOKUP_ROCKSDB_OPEN_FAILURE = "maha_lookup/rocksdb_open_failure"; - public static final String MAHA_LOOKUP_UPDATE_CACHE_SUCESS = "maha_lookup/update_cache_sucess"; + public static final String MAHA_LOOKUP_UPDATE_CACHE_SUCCESS = "maha_lookup/update_cache_success"; public static final String MAHA_LOOKUP_UPDATE_CACHE_FAILURE = "maha_lookup/update_cache_failure"; - public static final String MAHA_LOOKUP_PUBLISH_MISSING_LOOKUP_SUCESS = "maha_lookup/publish_missing_lookup_sucess"; + public static final String MAHA_LOOKUP_PUBLISH_MISSING_LOOKUP_SUCCESS = "maha_lookup/publish_missing_lookup_success"; public static final String MAHA_LOOKUP_PUBLISH_MISSING_LOOKUP_FAILURE = "maha_lookup/publish_missing_lookup_failure"; public static final String MAHA_LOOKUP_GET_CACHE_VALUE_FAILURE = "maha_lookup/get_cache_value_failure"; public static final String MAHA_LOOKUP_GET_CACHE_SIZE_FAILURE = "maha_lookup/get_cache_size_failure"; - public static final String MAHA_LOOKUP_GET_CACHE_VALUE_SUCESS = "maha_lookup/get_cache_value_sucess"; + public static final String MAHA_LOOKUP_GET_CACHE_VALUE_SUCCESS = "maha_lookup/get_cache_value_success"; public static final String MAHA_LOOKUP_ROCKSDB_INSTANCE_NOT_PRESENT = "maha_lookup/rocksdb_instance_not_present"; public static final String MAHA_LOOKUP_UPLOAD_LOOKUP_FOR_AUDITING_SUCCESS = "maha_lookup/upload_lookup_for_auditing_success"; public static final String MAHA_LOOKUP_UPLOAD_LOOKUP_FOR_AUDITING_FAILURE = "maha_lookup/upload_lookup_for_auditing_failure"; + public static final String MAHA_LOOKUP_MONGO_PROCESSING_TIME = "maha_lookup/mongo_processing_time"; + public static final String MAHA_LOOKUP_MONGO_CLIENT_FAILURE = "maha_lookup/mongo_client_failure"; + public static final String MAHA_LOOKUP_MONGO_DATABASE_OR_COLLECTION_FAILURE = "maha_lookup/mongo_database_or_collection_failure"; + public static final String MAHA_LOOKUP_MONGO_DOCUMENT_PROCESS_FAILURE = "maha_lookup/mongo_document_process_failure"; + public static final String MAHA_LOOKUP_NAME = "maha_lookup_name"; } diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java index 3a8d028c5..ed2dca3da 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/server/lookup/namespace/RocksDBExtractionNamespaceCacheFactory.java @@ -58,7 +58,7 @@ public String call() { public String call() { try { String loadTime = rocksDBManager.createDB(extractionNamespace, lastVersion); - emitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_ROCKSDB_OPEN_SUCESS, 1)); + emitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_ROCKSDB_OPEN_SUCCESS, 1)); return loadTime; } catch(Exception e) { LOG.error(e, "Caught exception while RocksDB creation, lastVersion: [%s]", lastVersion); @@ -99,7 +99,7 @@ public void updateCache(final RocksDBExtractionNamespace extractionNamespace, if(newLastUpdated > extractionNamespace.getLastUpdatedTime()) { extractionNamespace.setLastUpdatedTime(newLastUpdated); } - emitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_UPDATE_CACHE_SUCESS, 1)); + emitter.emit(ServiceMetricEvent.builder().build(MonitoringConstants.MAHA_LOOKUP_UPDATE_CACHE_SUCCESS, 1)); } } catch (Exception e) { LOG.error(e, "Caught exception while updating cache"); From d139675570b9a6c7ec6fa5546af6c056eb59e5a0 Mon Sep 17 00:00:00 2001 From: patelh Date: Mon, 25 Feb 2019 20:54:57 -0800 Subject: [PATCH 12/15] Update readme --- druid-lookups/README.md | 94 +++++++++++++++++++++++++++++++++++++++-- 1 file changed, 90 insertions(+), 4 deletions(-) diff --git a/druid-lookups/README.md b/druid-lookups/README.md index d2efa880f..bca6de40a 100644 --- a/druid-lookups/README.md +++ b/druid-lookups/README.md @@ -1,15 +1,101 @@ # Maha Druid Lookups -An extension to druid which provides for JDBC and RocksDB based lookups for high cardinality dimensions. For RocksDB, the lookups provide an interface to update entities via Kafka topics using the same protobuf format utilized for reading the RocksDB lookups. +An extension to druid which provides for MongoDB, JDBC and RocksDB (for high cardinality dimensions) based lookups. For RocksDB, the lookups provide an interface to update entities via Kafka topics using the same protobuf format utilized for reading the RocksDB lookups. The default behavior for RocksDB is to graba snapshot from HDFS daily and apply updates from Kafka from beginning of topic retention period. # Key Features -TODO +* Multi value lookups - Druid's default lookup extension only allows for a simple key/value pair. Our lookups allow you to have multiple columns for the value. At registration time, the spec defines the list of columns. At query time, the extractionFn definition provides which value column to render. +* High cardinality support - Druid's default lookup extension provides both off-heap and on-heap implementations for simple key/value pair. We extend this by allowing multiple columns and using RocksDB for on SSD lookups with off-heap LRU for high cardinality use cases. +* Provide a built in lookup service which is used to query lookups on historicals at query time for real-time tasks. Otherwise, real-time tasks would have to keep local copy of all lookups and that can get costly if they are high cardinality. ## Getting Started -TODO +Adding maha druid lookups to druid is simple. The command below will produce a zip file with all the jars in target directory which you can include in your druid distributions class path. + +``` +#this builds the jar and then assembles the zip file with all dependent jars +mvn clean install + +#the zip file can be found with this: +ls -l target/maha-druid-lookups-* + +-rw-r--r-- 1 patelh user 16084081 Feb 25 12:35 target/maha-druid-lookups-5.242-SNAPSHOT.zip + + +#now copy the zip file to your build server and unzip to get all the jars needed for the lookups and put them in druid jars directory. +``` + +### Configuration in common runtime.properties + +``` +# This is config for auth header factory, if you have your own implementation with whatever method you use to secure druid connections, set it here +druid.lookup.maha.namespace.authHeaderFactory=com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.NoopAuthHeaderFactory + +# This is your implementation of protobuf schema factory, only needed for RocksDB based lookups which require protobuf schema +druid.lookup.maha.namespace.schemaFactory=com.yahoo.maha.maha_druid_lookups.server.lookup.namespace.entity.NoopProtobufSchemaFactory + +# This is the scheme used by the lookup service, which is used by real-time nodes for looking up on historicals. Set this to https if using secured druid. +druid.lookup.maha.namespace.lookupService.service_scheme=http +druid.lookup.maha.namespace.lookupService.service_port=4080 + +# List of historicals which can be used by lookup service +druid.lookup.maha.namespace.lookupService.service_nodes=hist1,hist2,hist3 + +# Local storage directory for rocksdb based lookups +druid.lookup.maha.namespace.rocksdb.localStorageDirectory=/tmp + +# Block cache size for rocksdb based lookups +druid.lookup.maha.namespace.rocksdb.blockCacheSize=1048576 +``` ## Registering Druid Lookups -TODO +Druid lookups are managed using APIs on coordinators. Refer [here](http://druid.io/docs/latest/querying/lookups.html). + +Example POST to /druid/coordinator/v1/lookups/config : + +``` +{ + "__default": { + "advertiser_lookup": { + "version": "v1", + "lookupExtractorFactory": { + "type": "cachedNamespace", + "firstCacheTimeout": 180000, + "injective": false, + "extractionNamespace": { + "type": "mahamongo", + "connectorConfig": { + "hosts": "1.1.1.1:1111,1.1.1.2:1111", + "dbName": "mydb", + "user": "myuser", + "password": { + "type": "default", + "password": "mypassword" + }, + "clientOptions": { + "socketTimeout": 180000 + } + }, + "collectionName": "advertiser", + "tsColumn": "updated_at", + "tsColumnEpochInteger": true, + "pollPeriod": "PT60S", + "cacheEnabled": true, + "lookupName": "advertiser_lookup", + "documentProcessor": { + "type": "flatmultivalue", + "columnList": [ + "name", + "country", + "currency" + ], + "primaryKeyColumn": "_id" + }, + "mongoClientRetryCount": 3 + } + } + } + } +} +``` ## Auditing lookup integrity TODO From 3ab8e5838cc6313f6967c9aa0b3de5de997a3e19 Mon Sep 17 00:00:00 2001 From: patelh Date: Mon, 25 Feb 2019 21:09:57 -0800 Subject: [PATCH 13/15] Update readme --- druid-lookups/README.md | 53 ++++++++++++++++++++++++++++++++++++++--- 1 file changed, 50 insertions(+), 3 deletions(-) diff --git a/druid-lookups/README.md b/druid-lookups/README.md index bca6de40a..3dfc8fd16 100644 --- a/druid-lookups/README.md +++ b/druid-lookups/README.md @@ -1,14 +1,14 @@ # Maha Druid Lookups -An extension to druid which provides for MongoDB, JDBC and RocksDB (for high cardinality dimensions) based lookups. For RocksDB, the lookups provide an interface to update entities via Kafka topics using the same protobuf format utilized for reading the RocksDB lookups. The default behavior for RocksDB is to graba snapshot from HDFS daily and apply updates from Kafka from beginning of topic retention period. +An extension to druid which provides for MongoDB, JDBC and RocksDB (for high cardinality dimensions) based lookups. For RocksDB, the lookups provide an interface to update entities via Kafka topics using the same protobuf format utilized for reading the RocksDB lookups. The default behavior for RocksDB is to grab a snapshot from HDFS daily and apply updates from Kafka from beginning of topic retention period. # Key Features * Multi value lookups - Druid's default lookup extension only allows for a simple key/value pair. Our lookups allow you to have multiple columns for the value. At registration time, the spec defines the list of columns. At query time, the extractionFn definition provides which value column to render. * High cardinality support - Druid's default lookup extension provides both off-heap and on-heap implementations for simple key/value pair. We extend this by allowing multiple columns and using RocksDB for on SSD lookups with off-heap LRU for high cardinality use cases. -* Provide a built in lookup service which is used to query lookups on historicals at query time for real-time tasks. Otherwise, real-time tasks would have to keep local copy of all lookups and that can get costly if they are high cardinality. +* Lookup service for real-time tasks - Provides a built in lookup service which is used to query lookups on historicals at query time for real-time tasks. Otherwise, real-time tasks would have to keep local copy of all lookups and that can get costly if they are high cardinality. ## Getting Started -Adding maha druid lookups to druid is simple. The command below will produce a zip file with all the jars in target directory which you can include in your druid distributions class path. +Adding maha druid lookups to druid is simple. The command below will produce a zip file with all the jars in target directory which you can include in your druid installation's class path. ``` #this builds the jar and then assembles the zip file with all dependent jars @@ -34,6 +34,8 @@ druid.lookup.maha.namespace.schemaFactory=com.yahoo.maha.maha_druid_lookups.serv # This is the scheme used by the lookup service, which is used by real-time nodes for looking up on historicals. Set this to https if using secured druid. druid.lookup.maha.namespace.lookupService.service_scheme=http + +# This is the port your historicals are configured to use, needed by lookup service druid.lookup.maha.namespace.lookupService.service_port=4080 # List of historicals which can be used by lookup service @@ -97,6 +99,51 @@ Example POST to /druid/coordinator/v1/lookups/config : } ``` +### Example queries using above lookup + +Example query using lookup in filter, we filter advertisers by doing a lookup on country: + +``` +{ + "dataSource": "advertiser_stats", + "queryType": "topN", + "aggregations": [{ + "type": "doubleSum", + "name": "spend" + }], + "context": { + "timeout": 30000, + "queryId": "abcd" + }, + "filter": { + "type": "and", + "fields": [{ + "type": "selector", + "dimension": "advertiser_id", + "value": "US", + "extractionFn": { + "type": "mahaRegisteredLookup", + "lookup": "advertiser_lookup", + "retainMissingValue": false, + "replaceMissingValueWith": "N/A", + "injective": false, + "optimize": true, + "valueColumn": "country", + "dimensionOverrideMap": {}, + "useQueryLevelCache": false + } + }] + }, + "granularity": "all", + "threshold": 10, + "metric": "spend", + "dimension": "advertiser_id", + "intervals": [ + "2019-02-25T00:00:00.000Z/2019-02-26T00:00:00.000Z" + ] +} +``` + ## Auditing lookup integrity TODO From fc44439793f5e116b61896c4c498fbcc804c09d8 Mon Sep 17 00:00:00 2001 From: patelh Date: Thu, 28 Feb 2019 12:52:32 -0800 Subject: [PATCH 14/15] releasing next revision --- api-example/pom.xml | 2 +- api-jersey/pom.xml | 2 +- core/pom.xml | 2 +- db/pom.xml | 2 +- druid-lookups/pom.xml | 5 ++--- druid/pom.xml | 2 +- job-service/pom.xml | 2 +- oracle/pom.xml | 2 +- par-request-2/pom.xml | 2 +- pom.xml | 2 +- presto/pom.xml | 2 +- request-log/pom.xml | 2 +- service/pom.xml | 2 +- worker/pom.xml | 2 +- 14 files changed, 15 insertions(+), 16 deletions(-) diff --git a/api-example/pom.xml b/api-example/pom.xml index 8dbe944e4..8dc1948a0 100644 --- a/api-example/pom.xml +++ b/api-example/pom.xml @@ -11,7 +11,7 @@ maha-parent com.yahoo.maha - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha api-example diff --git a/api-jersey/pom.xml b/api-jersey/pom.xml index a628def7d..c3533e1c2 100644 --- a/api-jersey/pom.xml +++ b/api-jersey/pom.xml @@ -10,7 +10,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha api-jersey diff --git a/core/pom.xml b/core/pom.xml index f36845c21..e9d63c214 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -8,7 +8,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha core diff --git a/db/pom.xml b/db/pom.xml index 1601b72e7..770f9dc65 100644 --- a/db/pom.xml +++ b/db/pom.xml @@ -8,7 +8,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha db diff --git a/druid-lookups/pom.xml b/druid-lookups/pom.xml index 22b011f7f..59b6b0d46 100644 --- a/druid-lookups/pom.xml +++ b/druid-lookups/pom.xml @@ -1,12 +1,11 @@ - + maha-parent com.yahoo.maha - 5.242-SNAPSHOT + 5.243-SNAPSHOT 4.0.0 diff --git a/druid/pom.xml b/druid/pom.xml index 62f995bf7..795dd6f29 100644 --- a/druid/pom.xml +++ b/druid/pom.xml @@ -10,7 +10,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha druid executor diff --git a/job-service/pom.xml b/job-service/pom.xml index a9acf7ed5..5b3f672aa 100644 --- a/job-service/pom.xml +++ b/job-service/pom.xml @@ -8,7 +8,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha job service diff --git a/oracle/pom.xml b/oracle/pom.xml index 6b081490a..91bfc8c7d 100644 --- a/oracle/pom.xml +++ b/oracle/pom.xml @@ -10,7 +10,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha oracle executor diff --git a/par-request-2/pom.xml b/par-request-2/pom.xml index e37388925..294c339f1 100644 --- a/par-request-2/pom.xml +++ b/par-request-2/pom.xml @@ -9,7 +9,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT diff --git a/pom.xml b/pom.xml index 43f504ef9..617d1062c 100644 --- a/pom.xml +++ b/pom.xml @@ -5,7 +5,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT pom maha parent diff --git a/presto/pom.xml b/presto/pom.xml index a466e547a..c07ea9767 100644 --- a/presto/pom.xml +++ b/presto/pom.xml @@ -10,7 +10,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha presto executor diff --git a/request-log/pom.xml b/request-log/pom.xml index 329999392..4cf1fc9da 100644 --- a/request-log/pom.xml +++ b/request-log/pom.xml @@ -8,7 +8,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha request log diff --git a/service/pom.xml b/service/pom.xml index 9ca12c666..47a7599da 100644 --- a/service/pom.xml +++ b/service/pom.xml @@ -9,7 +9,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha service diff --git a/worker/pom.xml b/worker/pom.xml index 7adbb3f90..8c02764bb 100644 --- a/worker/pom.xml +++ b/worker/pom.xml @@ -8,7 +8,7 @@ com.yahoo.maha maha-parent - 5.242-SNAPSHOT + 5.243-SNAPSHOT maha worker From e71af0dc5f53a6a67254a6f46c7b108d56ab5850 Mon Sep 17 00:00:00 2001 From: patelh Date: Fri, 1 Mar 2019 11:01:14 -0800 Subject: [PATCH 15/15] Add backwards compatibility for inmemorydb + tests --- .../lookup/namespace/ExtractionNamespace.java | 1 + .../RocksDBExtractionNamespaceTest.java | 69 +++++++++++++++++++ .../inmemorydb_extraction_namespace.json | 9 +++ .../rocksdb_extraction_namespace.json | 9 +++ 4 files changed, 88 insertions(+) create mode 100644 druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespaceTest.java create mode 100644 druid-lookups/src/test/resources/inmemorydb_extraction_namespace.json create mode 100644 druid-lookups/src/test/resources/rocksdb_extraction_namespace.json diff --git a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java index 1b41a9d7c..f32791945 100644 --- a/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java +++ b/druid-lookups/src/main/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/ExtractionNamespace.java @@ -10,6 +10,7 @@ @JsonSubTypes(value = { @JsonSubTypes.Type(name = "mahajdbc", value = JDBCExtractionNamespace.class), @JsonSubTypes.Type(name = "maharocksdb", value = RocksDBExtractionNamespace.class), + @JsonSubTypes.Type(name = "mahainmemorydb", value = RocksDBExtractionNamespace.class), @JsonSubTypes.Type(name = "mahamongo", value = MongoExtractionNamespace.class) }) public interface ExtractionNamespace { diff --git a/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespaceTest.java b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespaceTest.java new file mode 100644 index 000000000..2435c2366 --- /dev/null +++ b/druid-lookups/src/test/java/com/yahoo/maha/maha_druid_lookups/query/lookup/namespace/RocksDBExtractionNamespaceTest.java @@ -0,0 +1,69 @@ +package com.yahoo.maha.maha_druid_lookups.query.lookup.namespace; + +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.ser.std.ToStringSerializer; +import com.fasterxml.jackson.datatype.joda.deser.DurationDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.PeriodDeserializer; +import com.fasterxml.jackson.datatype.joda.deser.key.DateTimeKeyDeserializer; +import com.google.common.collect.Lists; +import org.joda.time.DateTime; +import org.joda.time.Duration; +import org.joda.time.Interval; +import org.joda.time.Period; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; + +public class RocksDBExtractionNamespaceTest { + private static final ObjectMapper objectMapper = new ObjectMapper(); + + + @BeforeClass + void setup() { + SimpleModule module = new SimpleModule(); + module.addKeyDeserializer(DateTime.class, new DateTimeKeyDeserializer()); + module.addSerializer(DateTime.class, ToStringSerializer.instance); + module.addSerializer(Interval.class, ToStringSerializer.instance); + JsonDeserializer periodDeserializer = new PeriodDeserializer(); + module.addDeserializer(Period.class, (JsonDeserializer) periodDeserializer); + module.addSerializer(Period.class, ToStringSerializer.instance); + module.addDeserializer(Duration.class, new DurationDeserializer()); + module.addSerializer(Duration.class, ToStringSerializer.instance); + + objectMapper.registerModule(module); + + Period period = Period.seconds(30); + System.out.println(period); + } + + @Test + public void successfullyDeserializeFullNamespaceFromJSON() throws Exception { + RocksDBExtractionNamespace namespace = objectMapper + .readValue(ClassLoader.getSystemClassLoader().getResourceAsStream("rocksdb_extraction_namespace.json") + , RocksDBExtractionNamespace.class); + assertEquals(namespace.getNamespace(), "advertiser"); + assertEquals(namespace.getRocksDbInstanceHDFSPath(), "/data/druid/lookups/snapshots/advertiser"); + assertEquals(namespace.getLookupAuditingHDFSPath(), "/data/druid/lookups/audits/advertiser"); + assertEquals(namespace.getPollMs(), 30000); + assertEquals(namespace.isCacheEnabled(), true); + assertEquals(namespace.getLookupName(), "advertiser_lookup"); + + } + + @Test + public void successfullyDeserializeFullNamespaceFromInMemoryDbJSON() throws Exception { + RocksDBExtractionNamespace namespace = objectMapper + .readValue(ClassLoader.getSystemClassLoader().getResourceAsStream("inmemorydb_extraction_namespace.json") + , RocksDBExtractionNamespace.class); + assertEquals(namespace.getNamespace(), "advertiser"); + assertEquals(namespace.getRocksDbInstanceHDFSPath(), "/data/druid/lookups/snapshots/advertiser"); + assertEquals(namespace.getLookupAuditingHDFSPath(), "/data/druid/lookups/audits/advertiser"); + assertEquals(namespace.getPollMs(), 30000); + assertEquals(namespace.isCacheEnabled(), true); + assertEquals(namespace.getLookupName(), "advertiser_lookup"); + + } +} diff --git a/druid-lookups/src/test/resources/inmemorydb_extraction_namespace.json b/druid-lookups/src/test/resources/inmemorydb_extraction_namespace.json new file mode 100644 index 000000000..cc7f3858a --- /dev/null +++ b/druid-lookups/src/test/resources/inmemorydb_extraction_namespace.json @@ -0,0 +1,9 @@ +{ + "type": "mahainmemorydb", + "namespace": "advertiser", + "rocksDbInstanceHDFSPath": "/data/druid/lookups/snapshots/advertiser", + "lookupAuditingHDFSPath": "/data/druid/lookups/audits/advertiser", + "pollPeriod": "PT30S", + "cacheEnabled": true, + "lookupName": "advertiser_lookup" +} \ No newline at end of file diff --git a/druid-lookups/src/test/resources/rocksdb_extraction_namespace.json b/druid-lookups/src/test/resources/rocksdb_extraction_namespace.json new file mode 100644 index 000000000..e94dc8b04 --- /dev/null +++ b/druid-lookups/src/test/resources/rocksdb_extraction_namespace.json @@ -0,0 +1,9 @@ +{ + "type": "maharocksdb", + "namespace": "advertiser", + "rocksDbInstanceHDFSPath": "/data/druid/lookups/snapshots/advertiser", + "lookupAuditingHDFSPath": "/data/druid/lookups/audits/advertiser", + "pollPeriod": "PT30S", + "cacheEnabled": true, + "lookupName": "advertiser_lookup" +} \ No newline at end of file