You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2020/02/05 02:54:44 UTC

[lucene-solr] branch jira/SOLR-13101 updated: SOLR-13101: Convert nanotime to ms (#1208)

This is an automated email from the ASF dual-hosted git repository.

yonik pushed a commit to branch jira/SOLR-13101
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/SOLR-13101 by this push:
     new 51bf98c  SOLR-13101: Convert nanotime to ms (#1208)
51bf98c is described below

commit 51bf98cf8367a67d5d17fec6e172625967221cb4
Author: ebehrendt <31...@users.noreply.github.com>
AuthorDate: Tue Feb 4 18:54:33 2020 -0800

    SOLR-13101: Convert nanotime to ms (#1208)
    
    * Convert nanoseconds to milliseconds
    
    * Clean up from incorrect merge
    
    * Incorporate CR feedback to move time to a utility function. Use TimeUnit java util to convert from nanoseconds to milliseconds.
    
    * Rename method from getCurrentNanoTimeInMs to getCurrentTimeMs
---
 .../solr/store/blob/client/BlobCoreMetadata.java     |  3 +++
 .../solr/store/blob/metadata/CorePushPull.java       | 20 ++++++++++----------
 .../solr/store/blob/process/BlobDeleterTask.java     | 11 ++++++-----
 .../apache/solr/store/blob/process/CorePullTask.java |  4 ++--
 .../solr/store/blob/process/CorePullerFeeder.java    |  5 +++--
 .../apache/solr/store/blob/process/CorePusher.java   |  4 ++--
 .../apache/solr/store/blob/util/BlobStoreUtils.java  | 10 ++++++++++
 .../store/blob/client/CoreStorageClientTest.java     |  3 ++-
 8 files changed, 38 insertions(+), 22 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
index b5556d3..de12de0 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
@@ -324,6 +324,9 @@ public class BlobCoreMetadata {
       return Objects.hash(super.hashCode(), deletedAt);
     }
 
+    /**
+     * @return time in milliseconds (converted from nanotime) that file was marked as deleted
+     */
     public long getDeletedAt() {
       return this.deletedAt;
     }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
index 8a9a515..c659b9b 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
@@ -113,7 +113,7 @@ public class CorePushPull {
      * @param newMetadataSuffix suffix of the new core.metadata file to be created as part of this push
      */
     public BlobCoreMetadata pushToBlobStore(String currentMetadataSuffix, String newMetadataSuffix) throws Exception {
-      long startTimeMs = System.nanoTime();
+      long startTimeMs = BlobStoreUtils.getCurrentTimeMs();
       SolrCore solrCore = container.getCore(pushPullData.getCoreName());
       if (solrCore == null) {
         throw new Exception("Can't find core " + pushPullData.getCoreName());
@@ -137,7 +137,7 @@ public class CorePushPull {
          */
         for (BlobCoreMetadata.BlobFile d : resolvedMetadataResult.getFilesToDelete()) {
             bcmBuilder.removeFile(d);
-            BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete(d, System.currentTimeMillis());
+            BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete(d, BlobStoreUtils.getCurrentTimeMs());
             bcmBuilder.addFileToDelete(bftd);
         }
 
@@ -152,7 +152,7 @@ public class CorePushPull {
           String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(currentMetadataSuffix);
           String coreMetadataPath = blobMetadata.getSharedBlobName() + "/" + blobCoreMetadataName;
           // so far checksum is not used for metadata file
-          BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete("", coreMetadataPath, bcmSize, BlobCoreMetadataBuilder.UNDEFINED_VALUE, System.currentTimeMillis());
+          BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete("", coreMetadataPath, bcmSize, BlobCoreMetadataBuilder.UNDEFINED_VALUE, BlobStoreUtils.getCurrentTimeMs());
           bcmBuilder.addFileToDelete(bftd);
         }
 
@@ -191,21 +191,21 @@ public class CorePushPull {
         return newBcm;
       } finally {
         solrCore.close();
-        
+
         counter.setExpectedFilesTransferred(resolvedMetadataResult.getFilesToPush().size());
         logBlobAction("PUSH", counter, isSuccessful, startTimeMs, 0, startTimeMs);
       }
     }
 
     /**
-     * Calls {@link #pullUpdateFromBlob(long, boolean, int)}  with current epoch time and attempt no. 0.
+     * Calls {@link #pullUpdateFromBlob(long, boolean, int)}  with current relative time and attempt no. 0.
      * @param waitForSearcher <code>true</code> if this call should wait until the index searcher is created (so that any query
      *                     after the return from this method sees the new pulled content) or <code>false</code> if we request
      *                     a new index searcher to be eventually created but do not wait for it to be created (a query
      *                     following the return from this call might see the old core content).
      */
     public void pullUpdateFromBlob(boolean waitForSearcher) throws Exception {
-         pullUpdateFromBlob(System.nanoTime(), waitForSearcher, 0);
+         pullUpdateFromBlob(BlobStoreUtils.getCurrentTimeMs(), waitForSearcher, 0);
     }
 
     /**
@@ -217,7 +217,7 @@ public class CorePushPull {
      * <li>Local core did not exist (was created empty before calling this method) and is fetched from Blob</li>
      * </ol>
      * 
-     * @param requestQueuedTimeMs epoch time in milliseconds when the pull request was queued(meaningful in case of async pushing)
+     * @param requestQueuedTimeMs relative time in milliseconds when the pull request was queued(meaningful in case of async pushing)
      *                            only used for logging purposes
      * @param waitForSearcher <code>true</code> if this call should wait until the index searcher is created (so that any query
      *                     after the return from this method sees the new pulled content) or <code>false</code> if we request
@@ -232,7 +232,7 @@ public class CorePushPull {
      *                      TODO This has to be revisited before going to real prod, as environment issues can cause massive reindexing with this strategy
      */
     public void pullUpdateFromBlob(long requestQueuedTimeMs, boolean waitForSearcher, int attempt) throws Exception {
-        long startTimeMs = System.nanoTime();
+        long startTimeMs = BlobStoreUtils.getCurrentTimeMs();
         boolean isSuccessful = false;
         FileTransferCounter counter = new FileTransferCounter();
 
@@ -429,7 +429,7 @@ public class CorePushPull {
      */
     private void logBlobAction(String action, FileTransferCounter counter, boolean isSuccessful, long requestQueuedTimeMs, int attempt,
         long startTimeMs) throws Exception {
-      long now = System.nanoTime();
+      long now = BlobStoreUtils.getCurrentTimeMs();
       long runTime = now - startTimeMs;
       long startLatency = now - requestQueuedTimeMs;
 
@@ -507,7 +507,7 @@ public class CorePushPull {
     @VisibleForTesting
     protected boolean okForHardDelete(BlobCoreMetadata.BlobFileToDelete file) {
       // For now we only check how long ago the file was marked for delete.
-      return System.nanoTime() - file.getDeletedAt() >= deleteManager.getDeleteDelayMs();
+      return BlobStoreUtils.getCurrentTimeMs() - file.getDeletedAt() >= deleteManager.getDeleteDelayMs();
     }
     
     @VisibleForTesting
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleterTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleterTask.java
index f92d9aa..c2eaa53 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleterTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleterTask.java
@@ -25,6 +25,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.solr.store.blob.client.CoreStorageClient;
+import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -57,12 +58,12 @@ class BlobDeleterTask implements Runnable {
     this.blobNames = blobNames;
     this.attempt = new AtomicInteger(0);
     this.executor = executor;
-    this.queuedTimeMs = System.nanoTime();
+    this.queuedTimeMs = BlobStoreUtils.getCurrentTimeMs();
   }
 
   @Override
   public void run() {
-    final long startTimeMs = System.nanoTime();
+    final long startTimeMs = BlobStoreUtils.getCurrentTimeMs();
     boolean isSuccess = true;
       
     try {
@@ -97,14 +98,14 @@ class BlobDeleterTask implements Runnable {
           executor.execute(this);
         }
       } finally {
-        long now = System.nanoTime();
-        long runTime = now - startTimeMs;
+        long now = BlobStoreUtils.getCurrentTimeMs();
+        long runTimeMs = now - startTimeMs;
         long startLatency = now - this.queuedTimeMs;
         String message = String.format(Locale.ROOT,
                "sharedBlobName=%s action=DELETE storageProvider=%s bucketRegion=%s bucketName=%s "
                       + "runTime=%s startLatency=%s attempt=%s filesAffected=%s isSuccess=%s",
                       sharedBlobName, client.getStorageProvider().name(), client.getBucketRegion(),
-                      client.getBucketName(), runTime, startLatency, attempt.get(), this.blobNames.size(), isSuccess);
+                      client.getBucketName(), runTimeMs, startLatency, attempt.get(), this.blobNames.size(), isSuccess);
         log.info(message);
       }
   }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
index 921cad4..956741d 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
@@ -79,7 +79,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
   private final PullCoreCallback callback;
 
   CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, PullCoreCallback callback, Set<String> coresCreatedNotPulledYet) {
-    this(coreContainer, pullCoreInfo, System.nanoTime(), 0, 0L, callback, coresCreatedNotPulledYet);
+    this(coreContainer, pullCoreInfo, BlobStoreUtils.getCurrentTimeMs(), 0, 0L, callback, coresCreatedNotPulledYet);
   }
 
   @VisibleForTesting
@@ -207,7 +207,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
     final long lastAttemptTimestampCopy = getLastAttemptTimestamp();
 
     if (attemptsCopy != 0) {
-      long now = System.nanoTime();
+      long now = BlobStoreUtils.getCurrentTimeMs();
       if (now - lastAttemptTimestampCopy < MIN_RETRY_DELAY_MS) {
         Thread.sleep(MIN_RETRY_DELAY_MS - now + lastAttemptTimestampCopy);
       }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
index 56c94f4..93c07ce 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
@@ -25,6 +25,7 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.metadata.BlobCoreSyncer;
 import org.apache.solr.store.blob.metadata.PushPullData;
+import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.apache.solr.store.blob.util.DeduplicatingList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -116,7 +117,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       syncsEnqueuedSinceLastLog++;
 
       // Log if it's time (we did at least one pull otherwise we would be still blocked in the calls above)
-      final long now = System.nanoTime();
+      final long now = BlobStoreUtils.getCurrentTimeMs();
       final long msSinceLastLog = now - lastLoggedTimestamp;
       if (msSinceLastLog > minMsBetweenLogs) {
         log.info("Since last pull log " + msSinceLastLog + " ms ago, added "
@@ -212,7 +213,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
         PullCoreInfo pullCoreInfo = pullTask.getPullCoreInfo();
         if (status.isTransientError() && pullTask.getAttempts() < MAX_ATTEMPTS) {
           pullTask.setAttempts(pullTask.getAttempts() + 1);
-          pullTask.setLastAttemptTimestamp(System.nanoTime());
+          pullTask.setLastAttemptTimestamp(BlobStoreUtils.getCurrentTimeMs());
           pullTaskQueue.addDeduplicated(pullTask, true);
           log.info(String.format(Locale.ROOT, "Pulling core %s failed with transient error. Retrying. Last status=%s attempts=%s . %s",
               pullCoreInfo.getSharedStoreName(), status, pullTask.getAttempts(), message == null ? "" : message));
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java
index b7883d5..c30fbec 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java
@@ -90,10 +90,10 @@ public class CorePusher {
       // so there is no question of starvation.
       // One option could be to snapshot a queue of pusher threads, we are working on behalf of,
       // before we capture the commit point to push. Once finished pushing, we can dismiss all those threads together.
-      long startTimeMs = System.nanoTime();
+      long startTimeMs = BlobStoreUtils.getCurrentTimeMs();
       corePushLock.lock();
       try {
-        long lockAcquisitionTime = System.nanoTime() - startTimeMs;
+        long lockAcquisitionTime = BlobStoreUtils.getCurrentTimeMs() - startTimeMs;
         SolrCore core = coreContainer.getCore(coreName);
         if (core == null) {
           throw new SolrException(ErrorCode.SERVER_ERROR, "Can't find core " + coreName);
diff --git a/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java b/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
index 4a18da4..7eb1fb3 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
@@ -21,6 +21,7 @@ import java.util.HashMap;
 import java.util.Locale;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.ZkController;
@@ -75,6 +76,15 @@ public class BlobStoreUtils {
     return UUID.randomUUID().toString();
   }
 
+  /**
+   * Returns current time in milliseconds for use in measuring elapsed time.
+   * Cannot be combined with currentTimeMillis - currentTimeMillis will return ms relative to epoch
+   * while this method returns ms relative to some arbitrary time
+   */
+  public static long getCurrentTimeMs() {
+    return TimeUnit.MILLISECONDS.convert(System.nanoTime(), TimeUnit.NANOSECONDS);
+  }
+
   /***
    * syncLocalCoreWithSharedStore checks the local core has the latest copy stored in shared storage. Updates from the blob store always override the
    * local content.
diff --git a/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java b/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java
index b0533b1..d03da40 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java
@@ -29,6 +29,7 @@ import java.util.List;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -109,7 +110,7 @@ public class CoreStorageClientTest extends SolrTestCaseJ4 {
     File pulled = File.createTempFile("myPulledFile", ".txt");
     try {
       // Write binary data
-      byte bytesWritten[] = {0, -1, 5, 10, 32, 127, -15, 20, 0, -100, 40, 0, 0, 0, (byte) System.nanoTime()};
+      byte bytesWritten[] = {0, -1, 5, 10, 32, 127, -15, 20, 0, -100, 40, 0, 0, 0, (byte) BlobStoreUtils.getCurrentTimeMs()};
 
       FileUtils.writeByteArrayToFile(local, bytesWritten);