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 2020/06/24 08:35:23 UTC

[GitHub] [druid] clintropolis commented on a change in pull request #10033: Allow append to existing datasources when dynamic partitioning is used

clintropolis commented on a change in pull request #10033:
URL: https://github.com/apache/druid/pull/10033#discussion_r444696192



##########
File path: core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java
##########
@@ -73,5 +73,21 @@ public void testJsonPropertyNames() throws IOException
     Assert.assertEquals(expected.getPartitionDimensions(), map.get("partitionDimensions"));
     Assert.assertEquals(expected.getBucketId(), map.get("bucketId"));
     Assert.assertEquals(expected.getNumBuckets(), map.get("numPartitions"));
+    Assert.assertEquals(expected.getBucketId(), map.get("bucketId"));
+  }
+
+  @Test
+  public void testComplete()
+  {
+    final HashBasedNumberedPartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(
+        ImmutableList.of("dim"),
+        2,
+        4
+    );
+    final ShardSpec shardSpec = partialShardSpec.complete(new ObjectMapper(), 1, 3);

Review comment:
       why do this test use `new ObjectMapper()` instead of `MAPPER`?

##########
File path: indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionMultiPhaseParallelIndexingTest.java
##########
@@ -201,48 +208,122 @@ private static void writeRow(
   public void createsCorrectRangePartitions() throws Exception
   {
     int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION;
-    final Set<DataSegment> publishedSegments;
+    final Set<DataSegment> publishedSegments = runTestTask(
+        new SingleDimensionPartitionsSpec(
+            targetRowsPerSegment,
+            null,
+            DIM1,
+            false
+        ),
+        useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS,
+        false
+    );
+
+    if (!useMultivalueDim) {
+      assertRangePartitions(publishedSegments);
+    }
+  }
+
+  @Test
+  public void testAppendLinearlyPartitionedSegmensToHashPartitionedDatasourceSuccessfullyAppend()

Review comment:
       nit, typo: `Segmens` 
   "testAppendLinearlyPartitionedSegmentsToHashPartitionedDatasourceSuccessfullyAppend"

##########
File path: server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java
##########
@@ -142,6 +146,16 @@ public void testPossibleInDomain()
     Assert.assertTrue(shard7.possibleInDomain(domain2));
   }
 
+  @Test
+  public void testSharePartitionSpace()
+  {
+    final SingleDimensionShardSpec shardSpec = makeSpec("start", "end");
+    Assert.assertTrue(shardSpec.sharePartitionSpace(NumberedPartialShardSpec.instance()));
+    Assert.assertTrue(shardSpec.sharePartitionSpace(new HashBasedNumberedPartialShardSpec(null, 0, 1)));
+    Assert.assertTrue(shardSpec.sharePartitionSpace(new SingleDimensionPartialShardSpec("dim", 0, null, null, 1)));

Review comment:
       How do we end up with these mixed shard spec situations, minor compaction? Like I understand why these evaluate to true given how the checks work, just trying to understand how it happens since afaict only dynamic can be appended.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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



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