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/09/08 04:14:39 UTC

[GitHub] [druid] asdf2014 commented on a change in pull request #10363: fix injection failure of StorageLocationSelectorStrategy objects

asdf2014 commented on a change in pull request #10363:
URL: https://github.com/apache/druid/pull/10363#discussion_r484637958



##########
File path: server/src/main/java/org/apache/druid/segment/loading/LeastBytesUsedStorageLocationSelectorStrategy.java
##########
@@ -32,11 +34,17 @@
 public class LeastBytesUsedStorageLocationSelectorStrategy implements StorageLocationSelectorStrategy
 {
   private static final Ordering<StorageLocation> ORDERING = Ordering.from(Comparator
-      .comparingLong(StorageLocation::currSizeBytes));
+                                                                              .comparingLong(StorageLocation::currSizeBytes));

Review comment:
       Please roll back useless code formatting.

##########
File path: server/src/main/java/org/apache/druid/segment/loading/RoundRobinStorageLocationSelectorStrategy.java
##########
@@ -32,19 +35,25 @@
  */
 public class RoundRobinStorageLocationSelectorStrategy implements StorageLocationSelectorStrategy
 {
-
-  private final List<StorageLocation> storageLocations;
+  private List<StorageLocation> storageLocations = null;

Review comment:
       The default values ​​of reference types in Java are all null, so assigning null here should be meaningless.

##########
File path: server/src/main/java/org/apache/druid/segment/loading/MostAvailableSizeStorageLocationSelectorStrategy.java
##########
@@ -32,12 +34,18 @@
 public class MostAvailableSizeStorageLocationSelectorStrategy implements StorageLocationSelectorStrategy
 {
   private static final Ordering<StorageLocation> ORDERING = Ordering.from(Comparator
-      .comparingLong(StorageLocation::availableSizeBytes)
-      .reversed());
+                                                                              .comparingLong(StorageLocation::availableSizeBytes)

Review comment:
       Please roll back useless code formatting.

##########
File path: server/src/main/java/org/apache/druid/segment/loading/RandomStorageLocationSelectorStrategy.java
##########
@@ -31,9 +34,15 @@
 public class RandomStorageLocationSelectorStrategy implements StorageLocationSelectorStrategy
 {
 
-  private final List<StorageLocation> storageLocations;
+  private List<StorageLocation> storageLocations = null;

Review comment:
       The default values ​​of reference types in Java are all null, so assigning null here should be meaningless.

##########
File path: server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java
##########
@@ -99,7 +99,10 @@ public SegmentLoaderLocalCacheManager(
           )
       );
     }
-    this.strategy = config.getStorageLocationSelectorStrategy(locations);
+
+    this.strategy = config.getStorageLocationSelectorStrategy();
+    this.strategy.setLocations(locations);
+    log.info("using storage location strategy: %s", this.strategy.getClass().getSimpleName());

Review comment:
       Generally, the first letter of the exception information needs to be capitalized. In addition, %s needs to be surrounded by square brackets. So please use `Using storage location strategy: [%s]`.

##########
File path: server/src/test/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategyTest.java
##########
@@ -256,4 +269,75 @@ public void testMostAvailableSizeLocationSelectorStrategy() throws Exception
     Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_1",
         localStorageFolder1, loc3.getPath());
   }
+
+  @Test
+  public void testDefaultSelectorStrategyConfig() throws Exception

Review comment:
       In the newly added test cases, the `throw Exception` statement needs to be removed to pass the travis CI.

##########
File path: server/src/test/java/org/apache/druid/segment/loading/StorageLocationSelectorStrategyTest.java
##########
@@ -256,4 +269,75 @@ public void testMostAvailableSizeLocationSelectorStrategy() throws Exception
     Assert.assertEquals("The next element of the iterator should point to path local_storage_folder_1",
         localStorageFolder1, loc3.getPath());
   }
+
+  @Test
+  public void testDefaultSelectorStrategyConfig() throws Exception
+  {
+    //no druid.segmentCache.locationSelectorStrategy.type specified
+    final Properties props = new Properties();
+    SegmentLoaderConfig loaderConfig = makeInjectorWithProperties(props).getInstance(SegmentLoaderConfig.class);
+    Assert.assertEquals(LeastBytesUsedStorageLocationSelectorStrategy.class,
+                        loaderConfig.getStorageLocationSelectorStrategy().getClass());
+  }
+
+  @Test
+  public void testRoundRobinSelectorStrategyConfig() throws Exception
+  {
+    final Properties props = new Properties();
+    props.put("druid.segmentCache.locationSelectorStrategy.type", "roundRobin");

Review comment:
       Although `Properties` inherits from `Hashtable` and can call the put method, it is recommended to use the setProperty method. In this way, you can forcibly restrict the incoming Key and Value of the String type to avoid an error when you call store or save again after passing in non-String data.




----------------------------------------------------------------
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