Skip to content
Merged
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import org.apache.hadoop.ozone.ClientVersion;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.util.Time;
import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
import org.apache.ratis.thirdparty.io.grpc.Status;
import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts;
Expand Down Expand Up @@ -571,7 +572,7 @@ public XceiverClientReply sendCommandAsync(
final CompletableFuture<ContainerCommandResponseProto> replyFuture =
new CompletableFuture<>();
semaphore.acquire();
long requestTime = System.currentTimeMillis();
long requestTime = Time.monotonicNow();
metrics.incrPendingContainerOpsMetrics(request.getCmdType());

// create a new grpc message stream pair for each call.
Expand Down Expand Up @@ -601,7 +602,7 @@ public void onCompleted() {

private void decreasePendingMetricsAndReleaseSemaphore() {
metrics.decrPendingContainerOpsMetrics(request.getCmdType());
long cost = System.currentTimeMillis() - requestTime;
long cost = Time.monotonicNow() - requestTime;
metrics.addContainerOpsLatency(request.getCmdType(), cost);
if (LOG.isDebugEnabled()) {
LOG.debug("Executed command {} on datanode {}, cost = {}, cmdType = {}",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.security.SecurityConfig;
import org.apache.hadoop.hdds.tracing.TracingUtil;
import org.apache.hadoop.util.Time;
import org.apache.ratis.client.RaftClient;
import org.apache.ratis.client.api.DataStreamApi;
import org.apache.ratis.grpc.GrpcTlsConfig;
Expand Down Expand Up @@ -363,7 +364,7 @@ private XceiverClientReply handleFailedAllCommit(long index, Collection<CommitIn
public XceiverClientReply sendCommandAsync(
ContainerCommandRequestProto request) {
XceiverClientReply asyncReply = new XceiverClientReply(null);
long requestTime = System.currentTimeMillis();
long requestTime = Time.monotonicNow();
CompletableFuture<RaftClientReply> raftClientReply =
sendRequestAsync(request);
metrics.incrPendingContainerOpsMetrics(request.getCmdType());
Expand All @@ -376,7 +377,7 @@ public XceiverClientReply sendCommandAsync(
}
metrics.decrPendingContainerOpsMetrics(request.getCmdType());
metrics.addContainerOpsLatency(request.getCmdType(),
System.currentTimeMillis() - requestTime);
Time.monotonicNow() - requestTime);
}).thenApply(reply -> {
try {
if (!reply.isSuccess()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.hadoop.hdds.annotation.InterfaceStability;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -95,9 +96,9 @@ public void run() {
LOG.info("Shutdown process invoked a second time: ignoring");
return;
}
long started = System.currentTimeMillis();
long started = Time.monotonicNow();
int timeoutCount = MGR.executeShutdown();
long ended = System.currentTimeMillis();
long ended = Time.monotonicNow();
LOG.debug(String.format(
"Completed shutdown in %.3f seconds; Timeouts: %d",
(ended - started) / 1000.0, timeoutCount));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.ozone.container.metadata.DatanodeStore;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;

/**
Expand Down Expand Up @@ -89,7 +90,7 @@ public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
List<CompletableFuture<Void>> futures = new ArrayList<>();
List<HddsVolume> hddsVolumes = StorageVolumeUtil.getHddsVolumesList(
hddsVolumeSet.getVolumesList());
long start = System.currentTimeMillis();
long start = Time.monotonicNow();
for (HddsVolume volume : hddsVolumes) {
futures.add(CompletableFuture.runAsync(
() -> loadVolume(volume, readOnly, logger)));
Expand All @@ -99,7 +100,7 @@ public static void loadAllHddsVolumeDbStore(MutableVolumeSet hddsVolumeSet,
}
if (logger != null) {
logger.info("Load {} volumes DbStore cost: {}ms", hddsVolumes.size(),
System.currentTimeMillis() - start);
Time.monotonicNow() - start);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@
import org.apache.hadoop.ozone.container.replication.ReplicationServer.ReplicationConfig;
import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3;
import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.util.Timer;
import org.apache.ratis.grpc.GrpcTlsConfig;
import org.slf4j.Logger;
Expand Down Expand Up @@ -315,7 +316,7 @@ public void buildContainerSet() throws IOException {
Iterator<StorageVolume> volumeSetIterator = volumeSet.getVolumesList()
.iterator();
ArrayList<Thread> volumeThreads = new ArrayList<>();
long startTime = System.currentTimeMillis();
long startTime = Time.monotonicNow();

// Load container inspectors that may be triggered at startup based on
// system properties set. These can inspect and possibly repair
Expand Down Expand Up @@ -357,7 +358,7 @@ public void buildContainerSet() throws IOException {
ContainerInspectorUtil.unload();

LOG.info("Build ContainerSet costs {}s",
(System.currentTimeMillis() - startTime) / 1000);
(Time.monotonicNow() - startTime) / 1000);
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.util.Time;
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -464,7 +465,7 @@ public void testDeleteContainerTimeout() throws IOException {
final ContainerSet containerSet = newContainerSet();
final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class);
final Clock clock = mock(Clock.class);
long startTime = System.currentTimeMillis();
long startTime = Time.monotonicNow();

DatanodeConfiguration dnConf = conf.getObject(DatanodeConfiguration.class);
when(clock.millis())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.ozone.container.metadata.DatanodeStoreSchemaThreeImpl;
import org.apache.hadoop.util.Time;
import org.apache.ozone.test.GenericTestUtils.LogCapturer;
import org.apache.ratis.util.FileUtils;
import org.junit.jupiter.api.AfterEach;
Expand Down Expand Up @@ -450,15 +451,15 @@ public void testMultipleContainerReader(ContainerTestVersionInfo versionInfo)
(HddsVolume) volumes.get(i), containerSet, conf, true);
threads[i] = new Thread(containerReaders[i]);
}
long startTime = System.currentTimeMillis();
long startTime = Time.monotonicNow();
for (int i = 0; i < volumeNum; i++) {
threads[i].start();
}
for (int i = 0; i < volumeNum; i++) {
threads[i].join();
}
System.out.println("Open " + volumeNum + " Volume with " + containerCount +
" costs " + (System.currentTimeMillis() - startTime) / 1000 + "s");
" costs " + (Time.monotonicNow() - startTime) / 1000 + "s");
assertEquals(containerCount,
containerSet.getContainerMap().entrySet().size());
assertEquals(volumeSet.getFailedVolumesList().size(), 0);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import java.util.UUID;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
import org.apache.hadoop.util.Time;
import org.junit.jupiter.api.Test;

/**
Expand Down Expand Up @@ -104,7 +105,7 @@ public void test() throws InterruptedException {

ReplicationSupervisor rs = ReplicationSupervisor.newBuilder().build();

final long start = System.currentTimeMillis();
final long start = Time.monotonicNow();

//schedule 100 container replication
for (int i = 0; i < 100; i++) {
Expand All @@ -114,7 +115,7 @@ public void test() throws InterruptedException {
rs.addTask(new ReplicationTask(fromSources(i, sources), replicator));
}
rs.shutdownAfterFinish();
final long executionTime = System.currentTimeMillis() - start;
final long executionTime = Time.monotonicNow() - start;
System.out.println(executionTime);
assertThat(executionTime)
.withFailMessage("Execution was too slow : " + executionTime + " ms")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hadoop.ozone.lease.LeaseExpiredException;
import org.apache.hadoop.ozone.lease.LeaseManager;
import org.apache.hadoop.ozone.lease.LeaseNotFoundException;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -116,7 +117,7 @@ private synchronized void handleStartEvent(TIMEOUT_PAYLOAD payload,
EventPublisher publisher) {
metrics.incrementTrackedEvents();
long identifier = payload.getId();
startTrackingTimes.put(identifier, System.currentTimeMillis());
startTrackingTimes.put(identifier, Time.monotonicNow());

trackedEventsByID.put(identifier, payload);
trackedEvents.add(payload);
Expand All @@ -139,7 +140,7 @@ protected synchronized void handleCompletion(COMPLETION_PAYLOAD
if (trackedEvents.remove(payload)) {
metrics.incrementCompletedEvents();
long originalTime = startTrackingTimes.remove(id);
metrics.updateFinishingTime(System.currentTimeMillis() - originalTime);
metrics.updateFinishingTime(Time.monotonicNow() - originalTime);
onFinished(publisher, payload);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hadoop.metrics2.MetricsTag;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.lib.Interns;
import org.apache.hadoop.util.Time;
import org.apache.ratis.util.UncheckedAutoCloseable;

/**
Expand Down Expand Up @@ -68,12 +69,12 @@ public void increment(KEY key, long duration) {
}

public UncheckedAutoCloseable measure(KEY key) {
final long startTime = System.currentTimeMillis();
final long startTime = Time.monotonicNow();
concurrency.incrementAndGet();
return () -> {
concurrency.decrementAndGet();
counters.get(key).incrementAndGet();
elapsedTimes.get(key).addAndGet(System.currentTimeMillis() - startTime);
elapsedTimes.get(key).addAndGet(Time.monotonicNow() - startTime);
};
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.utils.MetadataKeyFilters;
import org.apache.hadoop.hdds.utils.db.RocksDatabase.ColumnFamily;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -297,7 +298,7 @@ private List<KeyValue<byte[], byte[]>> getRangeKVs(byte[] startKey,
int count, boolean sequential, byte[] prefix,
MetadataKeyFilters.MetadataKeyFilter... filters)
throws IOException, IllegalArgumentException {
long start = System.currentTimeMillis();
long start = Time.monotonicNow();

if (count < 0) {
throw new IllegalArgumentException(
Expand Down Expand Up @@ -342,7 +343,7 @@ && get(startKey) == null) {
}
}
} finally {
long end = System.currentTimeMillis();
long end = Time.monotonicNow();
long timeConsumed = end - start;
if (LOG.isDebugEnabled()) {
if (filters != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
import org.apache.hadoop.ozone.upgrade.UpgradeException.ResultCodes;
import org.apache.hadoop.ozone.upgrade.UpgradeFinalization.Status;
import org.apache.hadoop.ozone.upgrade.UpgradeFinalization.StatusAndMessages;
import org.apache.hadoop.util.Time;
import org.apache.ratis.protocol.exceptions.NotLeaderException;

/**
Expand Down Expand Up @@ -169,9 +170,9 @@ public void finalizeAndWaitForCompletion(
}

boolean success = false;
long endTime = System.currentTimeMillis() +
long endTime = Time.monotonicNow() +
TimeUnit.SECONDS.toMillis(maxTimeToWaitInSeconds);
while (System.currentTimeMillis() < endTime) {
while (Time.monotonicNow() < endTime) {
try {
response = reportStatus(upgradeClientID, false);
LOG.info("Finalization Messages : {} ", response.msgs());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@
import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksIterator;
import org.apache.hadoop.hdds.utils.db.managed.ManagedSstFileReader;
import org.apache.hadoop.ozone.lock.BootstrapStateHandler;
import org.apache.hadoop.util.Time;
import org.apache.ozone.compaction.log.CompactionFileInfo;
import org.apache.ozone.compaction.log.CompactionLogEntry;
import org.apache.ozone.rocksdb.util.RdbUtil;
Expand Down Expand Up @@ -758,7 +759,7 @@ void diffAllSnapshots(RocksDBCheckpointDiffer differ)
private void createCheckpoint(ManagedRocksDB rocksDB) throws RocksDBException {

LOG.trace("Current time: " + System.currentTimeMillis());
long t1 = System.currentTimeMillis();
long t1 = Time.monotonicNow();

final long snapshotGeneration = rocksDB.get().getLatestSequenceNumber();
final String cpPath = CP_PATH_PREFIX + snapshotGeneration;
Expand All @@ -780,7 +781,7 @@ private void createCheckpoint(ManagedRocksDB rocksDB) throws RocksDBException {
colHandle));
this.snapshots.add(currentSnapshot);

long t2 = System.currentTimeMillis();
long t2 = Time.monotonicNow();
LOG.trace("Current time: " + t2);
LOG.debug("Time elapsed: " + (t2 - t1) + " ms");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.common.base.Preconditions;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -105,7 +106,7 @@ public static int await(int timeoutMillis,
"timeoutMillis must be >= 0");
Preconditions.checkNotNull(timeoutHandler);

final long endTime = System.currentTimeMillis() + timeoutMillis;
final long endTime = Time.monotonicNow() + timeoutMillis;
Throwable ex = null;
boolean running = true;
int iterations = 0;
Expand All @@ -126,7 +127,7 @@ public static int await(int timeoutMillis,
LOG.debug("await() iteration {}", iterations, e);
ex = e;
}
running = System.currentTimeMillis() < endTime;
running = Time.monotonicNow() < endTime;
if (running) {
int sleeptime = retry.call();
if (sleeptime >= 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.util.Time;
import picocli.CommandLine.Command;
import picocli.CommandLine.Mixin;
import picocli.CommandLine.Option;
Expand Down Expand Up @@ -48,7 +49,7 @@ public class SafeModeWaitSubcommand implements Callable<Void> {

@Override
public Void call() throws Exception {
startTestTime = System.currentTimeMillis();
startTestTime = Time.monotonicNow();

while (getRemainingTimeInSec() > 0) {
try (ScmClient scmClient = scmOption.createScmClient()) {
Expand Down Expand Up @@ -85,6 +86,6 @@ public Void call() throws Exception {
}

private long getRemainingTimeInSec() {
return timeoutSeconds - (System.currentTimeMillis() - startTestTime) / 1000;
return timeoutSeconds - (Time.monotonicNow() - startTestTime) / 1000;
}
}
Loading