You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2018/08/17 15:54:08 UTC

[GitHub] gianm closed pull request #6186: Serde test for JdbcExtractionNamespace.

gianm closed pull request #6186: Serde test for JdbcExtractionNamespace.
URL: https://github.com/apache/incubator-druid/pull/6186
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java b/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java
index 4842cef78b5..c9d9cd2c771 100644
--- a/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java
+++ b/common/src/main/java/io/druid/metadata/MetadataStorageConnectorConfig.java
@@ -59,6 +59,7 @@ public int getPort()
     return port;
   }
 
+  @JsonProperty
   public String getConnectURI()
   {
     if (connectURI == null) {
diff --git a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java
index 8104e942eb3..007ba07a300 100644
--- a/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java
+++ b/extensions-core/lookups-cached-global/src/test/java/io/druid/server/lookup/namespace/cache/JdbcExtractionNamespaceTest.java
@@ -26,14 +26,16 @@
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import io.druid.common.config.NullHandling;
-import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.StringUtils;
+import io.druid.java.util.common.concurrent.Execs;
 import io.druid.java.util.common.io.Closer;
 import io.druid.java.util.common.lifecycle.Lifecycle;
 import io.druid.java.util.common.logger.Logger;
 import io.druid.metadata.TestDerbyConnector;
 import io.druid.query.lookup.namespace.CacheGenerator;
+import io.druid.query.lookup.namespace.ExtractionNamespace;
 import io.druid.query.lookup.namespace.JdbcExtractionNamespace;
+import io.druid.server.ServerTestHelper;
 import io.druid.server.lookup.namespace.JdbcCacheGenerator;
 import io.druid.server.lookup.namespace.NamespaceExtractionConfig;
 import io.druid.server.metrics.NoopServiceEmitter;
@@ -225,7 +227,11 @@ public void close()
                       }
                     }
                 ),
-                new OnHeapNamespaceExtractionCacheManager(lifecycle, noopServiceEmitter, new NamespaceExtractionConfig())
+                new OnHeapNamespaceExtractionCacheManager(
+                    lifecycle,
+                    noopServiceEmitter,
+                    new NamespaceExtractionConfig()
+                )
             );
             try {
               lifecycle.start();
@@ -382,7 +388,11 @@ public void testMappingWithoutFilter()
         String key = e.getKey();
         String[] val = e.getValue();
         String field = val[0];
-        Assert.assertEquals("non-null check", NullHandling.emptyToNullIfNeeded(field), NullHandling.emptyToNullIfNeeded(map.get(key)));
+        Assert.assertEquals(
+            "non-null check",
+            NullHandling.emptyToNullIfNeeded(field),
+            NullHandling.emptyToNullIfNeeded(map.get(key))
+        );
       }
       Assert.assertEquals("null check", null, map.get("baz"));
     }
@@ -412,7 +422,11 @@ public void testMappingWithFilter()
         String filterVal = val[1];
 
         if ("1".equals(filterVal)) {
-          Assert.assertEquals("non-null check", NullHandling.emptyToNullIfNeeded(field), NullHandling.emptyToNullIfNeeded(map.get(key)));
+          Assert.assertEquals(
+              "non-null check",
+              NullHandling.emptyToNullIfNeeded(field),
+              NullHandling.emptyToNullIfNeeded(map.get(key))
+          );
         } else {
           Assert.assertEquals("non-null check", null, NullHandling.emptyToNullIfNeeded(map.get(key)));
         }
@@ -457,6 +471,27 @@ public void testIgnoresNullValues()
     }
   }
 
+  @Test
+  public void testSerde() throws IOException
+  {
+    final JdbcExtractionNamespace extractionNamespace = new JdbcExtractionNamespace(
+        derbyConnectorRule.getMetadataConnectorConfig(),
+        tableName,
+        keyName,
+        valName,
+        tsColumn,
+        "some filter",
+        new Period(10)
+    );
+
+    final ExtractionNamespace extractionNamespace2 = ServerTestHelper.MAPPER.readValue(
+        ServerTestHelper.MAPPER.writeValueAsBytes(extractionNamespace),
+        ExtractionNamespace.class
+    );
+
+    Assert.assertEquals(extractionNamespace, extractionNamespace2);
+  }
+
   private CacheScheduler.Entry ensureEntry()
       throws InterruptedException
   {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org