frankgh commented on code in PR #234:
URL: https://github.com/apache/cassandra-sidecar/pull/234#discussion_r2180973759


##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/data/RestoreJobConstants.java:
##########
@@ -36,6 +36,17 @@ public class RestoreJobConstants
     public static final String JOB_CONSISTENCY_LEVEL = "consistencyLevel";
     public static final String JOB_OPERATION_REASON = "reason";
     public static final String JOB_LOCAL_DATA_CENTER = "localDatacenter";
+    /**
+     * A boolean field; when restoreToLocalDatacenterOnly is set to true, 
localDatacenter must also be specified, and
+     * the restore job will only restore to the specified localDatacenter, 
regardless of the replication configuration
+     * of the belonging keyspace.
+     * The option should be used with caution. To help build an educated 
decision, when only restoring to the local
+     * datacenter, but the table replicates to multiple datacenters, it could 
cause a large amount of repair streaming
+     * traffic. Argurably, you might want to leverage the intra-node repair 
feature, then you can enable this option.
+     * Another use case that could justify the option is runing distinct 
restore jobs, one per datacenter, concurrently.
+     * In this case, there is an external coordinator that manges the resotre 
job in each datacenter.

Review Comment:
   minor typos
   ```suggestion
        * traffic. Arguably, you might want to leverage the intra-node repair 
feature, then you can enable this option.
        * Another use case that could justify the option is running distinct 
restore jobs, one per datacenter, concurrently.
        * In this case, there is an external coordinator that manges the 
restore job in each datacenter.
   ```



##########
server/src/test/java/org/apache/cassandra/sidecar/restore/RestoreJobDiscovererTest.java:
##########
@@ -379,6 +385,59 @@ void testAdjustRecencyDays()
         .isEqualTo(10);
     }
 
+    @Test
+    void testSkipNotOwnedRestoreToLocalDatacenterOnlyJob()
+    {
+        // Create a restore job that restores to dc2 only. Meanwhile, 
discoverer runs in dc1.
+        UUID jobId = UUIDs.timeBased();
+        when(mockJobAccessor.findAllRecent(anyLong(), anyInt()))
+        .thenReturn(Collections.singletonList(RestoreJob.builder()
+                                                        
.createdAt(RestoreJob.toLocalDate(jobId))
+                                                        .jobId(jobId)
+                                                        .jobAgent("agent")
+                                                        
.jobStatus(RestoreJobStatus.CREATED)
+                                                        .expireAt(new 
Date(System.currentTimeMillis() + 10000L))
+                                                        
.consistencyLevel(ConsistencyLevel.LOCAL_QUORUM)
+                                                        .localDatacenter("dc2")
+                                                        
.shouldRestoreToLocalDatacenterOnly(true)
+                                                        .build()));
+        executeBlocking();
+        verify(mockRangeAccessor, never()).create(any());
+    }
+
+    @Test
+    void 
testRestoreToLocalDatacenterOnlyJobIsOnHoldWhenLocalDatacenterInUndetermined()
+    {
+        // local datacenter is undetermined and the restore job is configured 
to restore to local datacenter only.
+        // the job is on hold until local datacenter is resolved in dicoverer.
+        
when(instanceMetadataFetcher.callOnFirstAvailableInstance(any())).thenThrow(new 
CassandraUnavailableException(JMX, "NodeSettings unavailable"));
+        UUID jobId = UUIDs.timeBased();
+        when(mockJobAccessor.findAllRecent(anyLong(), anyInt()))
+        .thenReturn(Collections.singletonList(RestoreJob.builder()
+                                                        
.createdAt(RestoreJob.toLocalDate(jobId))
+                                                        .jobId(jobId)
+                                                        .jobAgent("agent")
+                                                        
.jobStatus(RestoreJobStatus.CREATED)
+                                                        .expireAt(new 
Date(System.currentTimeMillis() + 10000L))
+                                                        
.consistencyLevel(ConsistencyLevel.LOCAL_QUORUM)
+                                                        .localDatacenter("dc1")
+                                                        
.shouldRestoreToLocalDatacenterOnly(true)
+                                                        .build()));
+
+        executeBlocking();
+        verify(ringTopologyRefresher, never()).register(any(), any());
+
+        // in the new run, the local datacenter is discovered. The discoverer 
proceeds further and register the job

Review Comment:
   ```suggestion
           // in the new run, the local datacenter is discovered. The 
discoverer proceeds further and registers the job
   ```



##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RestoreJobDiscoverer.java:
##########
@@ -385,6 +399,41 @@ private void processSidecarManagedJobMaybe(RestoreJob job)
         }
     }
 
+    private void initLocalDatacenterMaybe()
+    {
+        if (localDatacenter != null)
+        {
+            return;
+        }
+
+        try
+        {
+            NodeSettings nodeSettings = 
instanceMetadataFetcher.callOnFirstAvailableInstance(i -> 
i.delegate().nodeSettings());
+            localDatacenter = nodeSettings.datacenter();
+        }
+        catch (CassandraUnavailableException cue)
+        {
+            LOGGER.debug("localDatacenter is not initialized", cue);
+        }
+    }
+
+    private boolean isLocalDatacenterExcluded(RestoreJob job)
+    {
+        if (!job.shouldRestoreToLocalDatacenterOnly)
+        {
+            return false;
+        }
+
+        if (localDatacenter == null)
+        {
+            LOGGER.debug("The restore job should restore only to the local 
datacenter, but the local datacetner is undetermined yet; skip this run");
+            return true;
+        }
+
+        // when job should restore to local datacenter only, but the target 
datacenter is not the local one
+        return !localDatacenter.equalsIgnoreCase(job.localDatacenter);

Review Comment:
   datacenter is case sensitive, so I don't think we should equal here with 
ignore case, even though it would be bad if someone decided to have 
`DATACENTER1` and `datacenter1`, but it would be possible



##########
server/src/test/java/org/apache/cassandra/sidecar/restore/RestoreJobDiscovererTest.java:
##########
@@ -379,6 +385,59 @@ void testAdjustRecencyDays()
         .isEqualTo(10);
     }
 
+    @Test
+    void testSkipNotOwnedRestoreToLocalDatacenterOnlyJob()
+    {
+        // Create a restore job that restores to dc2 only. Meanwhile, 
discoverer runs in dc1.
+        UUID jobId = UUIDs.timeBased();
+        when(mockJobAccessor.findAllRecent(anyLong(), anyInt()))
+        .thenReturn(Collections.singletonList(RestoreJob.builder()
+                                                        
.createdAt(RestoreJob.toLocalDate(jobId))
+                                                        .jobId(jobId)
+                                                        .jobAgent("agent")
+                                                        
.jobStatus(RestoreJobStatus.CREATED)
+                                                        .expireAt(new 
Date(System.currentTimeMillis() + 10000L))
+                                                        
.consistencyLevel(ConsistencyLevel.LOCAL_QUORUM)
+                                                        .localDatacenter("dc2")
+                                                        
.shouldRestoreToLocalDatacenterOnly(true)
+                                                        .build()));
+        executeBlocking();
+        verify(mockRangeAccessor, never()).create(any());
+    }
+
+    @Test
+    void 
testRestoreToLocalDatacenterOnlyJobIsOnHoldWhenLocalDatacenterInUndetermined()

Review Comment:
   Is this what you meant?
   ```suggestion
       void 
testRestoreToLocalDatacenterOnlyJobIsOnHoldWhenLocalDatacenterIsUndetermined()
   ```



##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/request/data/CreateRestoreJobRequestPayload.java:
##########
@@ -86,7 +89,8 @@ public CreateRestoreJobRequestPayload(@JsonProperty(JOB_ID) 
UUID jobId,
                                           @JsonProperty(JOB_IMPORT_OPTIONS) 
SSTableImportOptions importOptions,
                                           @JsonProperty(JOB_EXPIRE_AT) long 
expireAtInMillis,
                                           @JsonProperty(JOB_CONSISTENCY_LEVEL) 
String consistencyLevel,
-                                          @JsonProperty(JOB_LOCAL_DATA_CENTER) 
String localDatacenter)
+                                          @JsonProperty(JOB_LOCAL_DATA_CENTER) 
String localDatacenter,

Review Comment:
   let's add the missing parameters to javadocs



##########
server/src/main/java/org/apache/cassandra/sidecar/restore/RestoreJobDiscoverer.java:
##########
@@ -385,6 +399,41 @@ private void processSidecarManagedJobMaybe(RestoreJob job)
         }
     }
 
+    private void initLocalDatacenterMaybe()
+    {
+        if (localDatacenter != null)
+        {
+            return;
+        }
+
+        try
+        {
+            NodeSettings nodeSettings = 
instanceMetadataFetcher.callOnFirstAvailableInstance(i -> 
i.delegate().nodeSettings());
+            localDatacenter = nodeSettings.datacenter();
+        }
+        catch (CassandraUnavailableException cue)
+        {
+            LOGGER.debug("localDatacenter is not initialized", cue);
+        }
+    }
+
+    private boolean isLocalDatacenterExcluded(RestoreJob job)
+    {
+        if (!job.shouldRestoreToLocalDatacenterOnly)
+        {
+            return false;
+        }
+
+        if (localDatacenter == null)
+        {
+            LOGGER.debug("The restore job should restore only to the local 
datacenter, but the local datacetner is undetermined yet; skip this run");

Review Comment:
   ```suggestion
               LOGGER.debug("The restore job should restore only to the local 
datacenter, but the local datacenter is undetermined yet; skip this run");
   ```



##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/request/data/CreateRestoreJobRequestPayload.java:
##########
@@ -101,6 +105,9 @@ public CreateRestoreJobRequestPayload(@JsonProperty(JOB_ID) 
UUID jobId,
                              : importOptions;
         this.expireAtInMillis = expireAtInMillis;
         this.consistencyConfig = 
ConsistencyConfig.parseString(consistencyLevel, localDatacenter);
+        Preconditions.checkArgument(!localDatacenterOnly || 
StringUtils.isNotEmpty(localDatacenter),
+                                    "Must specify a non-empty localDatacenter 
when restoreToLocalDatacenterOnly is configured to true");

Review Comment:
   I think the `non-empty` part is implied
   ```suggestion
                                       "Must specify a localDatacenter when 
restoreToLocalDatacenterOnly is true");
   ```



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

To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org
For additional commands, e-mail: pr-h...@cassandra.apache.org

Reply via email to