You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/07/15 13:38:59 UTC

[GitHub] [iceberg] nastra opened a new pull request, #5286: API: Introduce DefaultMetricsContext and Timer interface

nastra opened a new pull request, #5286:
URL: https://github.com/apache/iceberg/pull/5286

   The idea here is that a `DefaultMetricsContext` can be used with native
   Java Counters/Timers to measure things.
   
   Additionally, this is introducing an API and a default implementation
   for a Timer, which can be used to measure execution time.


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#issuecomment-1189745817

   Looks great. Thanks for adding this, @nastra!


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r922678479


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {

Review Comment:
   We might want to make this a precondition check. I would assume if somebody is recording a negative time there's actually a problem.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r923092874


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      this.duration = this.duration.plus(amount, toChronoUnit(unit));

Review Comment:
   you are in fact absolutely right. I changed the implementation slightly and added a multi-threaded test (which was failing with the old implementation)



##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;

Review Comment:
   should be fixed now



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r922679921


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;

Review Comment:
   We might want to consider throwing if there are any running samples. It seems all too easy that something might be running in a thread and have not completed when total duration gets called. Then it wouldn't be included in the total.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #5286:
URL: https://github.com/apache/iceberg/pull/5286


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r923002528


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      this.duration = this.duration.plus(amount, toChronoUnit(unit));
+      count.incrementAndGet();
+    }
+  }
+
+  @Override
+  public <T> T record(Supplier<T> supplier) {
+    Sample sample = start();
+    try {
+      return supplier.get();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public <T> T recordCallable(Callable<T> callable) throws Exception {
+    Sample sample = start();
+    try {
+      return callable.call();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public void record(Runnable runnable) {
+    Sample sample = start();
+    try {
+      runnable.run();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("defaultTimeUnit", defaultTimeUnit)
+        .add("count", count)
+        .add("duration", duration)
+        .add("current", stopwatchRef)
+        .toString();
+  }
+
+  private static class DefaultSample implements Sample {
+    private final AtomicReference<Stopwatch> stopwatchRef;
+    private final TimeUnit defaultTimeUnit;
+
+    private DefaultSample(AtomicReference<Stopwatch> stopwatchRef, TimeUnit defaultTimeUnit) {
+      this.stopwatchRef = stopwatchRef;
+      this.defaultTimeUnit = defaultTimeUnit;
+    }
+
+    public void stop(Timer timer) {
+      Preconditions.checkArgument(null != timer, "Timer must be non-null");
+      Stopwatch stopwatch = stopwatchRef.getAndSet(null);
+      Preconditions.checkState(null != stopwatch, "startTimer() was not called");
+      timer.record(stopwatch.stop().elapsed(defaultTimeUnit), defaultTimeUnit);
+    }
+  }
+
+  /**
+   * Converts this {@code TimeUnit} to the equivalent {@code ChronoUnit}.
+   *
+   * @return the converted equivalent ChronoUnit
+   */
+  private static ChronoUnit toChronoUnit(TimeUnit timeUnit) {
+    switch (timeUnit) {

Review Comment:
   yeah this is only available in JDK9+, so I had to write up something for this



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r922179911


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultMetricsContext.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A default {@link MetricsContext} implementation that uses native Java counters/timers.
+ */
+public class DefaultMetricsContext implements MetricsContext {
+
+  @Override
+  public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
+    if (Integer.class.equals(type)) {
+      return (Counter<T>) new IntCounter();
+    }
+    if (Long.class.equals(type)) {
+      return (Counter<T>) new LongCounter();
+    }
+    throw new IllegalArgumentException(String.format("Counter for type %s is not supported", type.getName()));
+  }
+
+  @Override
+  public Timer timer(String name, TimeUnit unit) {

Review Comment:
   @danielcweeks I wasn't sure if it's worth adding a time unit to `MetricsContext.Unit`. I figured using Java's `TimeUnit` here is sufficient, but let me know if you think otherwise.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
nastra commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r923129700


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {

Review Comment:
   done



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r922678346


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      this.duration = this.duration.plus(amount, toChronoUnit(unit));
+      count.incrementAndGet();
+    }
+  }
+
+  @Override
+  public <T> T record(Supplier<T> supplier) {
+    Sample sample = start();
+    try {
+      return supplier.get();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public <T> T recordCallable(Callable<T> callable) throws Exception {
+    Sample sample = start();
+    try {
+      return callable.call();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public void record(Runnable runnable) {
+    Sample sample = start();
+    try {
+      runnable.run();
+    } finally {
+      sample.stop(this);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("defaultTimeUnit", defaultTimeUnit)
+        .add("count", count)
+        .add("duration", duration)
+        .add("current", stopwatchRef)
+        .toString();
+  }
+
+  private static class DefaultSample implements Sample {
+    private final AtomicReference<Stopwatch> stopwatchRef;
+    private final TimeUnit defaultTimeUnit;
+
+    private DefaultSample(AtomicReference<Stopwatch> stopwatchRef, TimeUnit defaultTimeUnit) {
+      this.stopwatchRef = stopwatchRef;
+      this.defaultTimeUnit = defaultTimeUnit;
+    }
+
+    public void stop(Timer timer) {
+      Preconditions.checkArgument(null != timer, "Timer must be non-null");
+      Stopwatch stopwatch = stopwatchRef.getAndSet(null);
+      Preconditions.checkState(null != stopwatch, "startTimer() was not called");
+      timer.record(stopwatch.stop().elapsed(defaultTimeUnit), defaultTimeUnit);
+    }
+  }
+
+  /**
+   * Converts this {@code TimeUnit} to the equivalent {@code ChronoUnit}.
+   *
+   * @return the converted equivalent ChronoUnit
+   */
+  private static ChronoUnit toChronoUnit(TimeUnit timeUnit) {
+    switch (timeUnit) {

Review Comment:
   It seems like there should be a better way to do this with the native libraries, but I see that wasn't introduced until Java 9.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r922677890


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultMetricsContext.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A default {@link MetricsContext} implementation that uses native Java counters/timers.
+ */
+public class DefaultMetricsContext implements MetricsContext {
+
+  @Override
+  public <T extends Number> Counter<T> counter(String name, Class<T> type, Unit unit) {
+    if (Integer.class.equals(type)) {
+      return (Counter<T>) new IntCounter();
+    }
+    if (Long.class.equals(type)) {
+      return (Counter<T>) new LongCounter();
+    }
+    throw new IllegalArgumentException(String.format("Counter for type %s is not supported", type.getName()));
+  }
+
+  @Override
+  public Timer timer(String name, TimeUnit unit) {

Review Comment:
   I think that's fine.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] danielcweeks commented on a diff in pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
danielcweeks commented on code in PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#discussion_r922679391


##########
api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.metrics;
+
+import java.time.Duration;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.base.Stopwatch;
+
+/**
+ * A default {@link Timer} implementation that uses a {@link Stopwatch} instance internally to measure time.
+ */
+public class DefaultTimer implements Timer {
+  private final TimeUnit defaultTimeUnit;
+  private final AtomicLong count = new AtomicLong();
+  private Duration duration = Duration.ZERO;
+  private final AtomicReference<Stopwatch> stopwatchRef = new AtomicReference<>();
+
+  public DefaultTimer(TimeUnit timeUnit) {
+    Preconditions.checkArgument(null != timeUnit, "TimeUnit must be non-null");
+    this.defaultTimeUnit = timeUnit;
+  }
+
+  @Override
+  public long count() {
+    return count.get();
+  }
+
+  @Override
+  public Duration totalDuration() {
+    return duration;
+  }
+
+  @Override
+  public Sample start() {
+    boolean updated = stopwatchRef.compareAndSet(null, Stopwatch.createStarted());
+    Preconditions.checkState(updated, "startTimer() called multiple times");
+    return new DefaultSample(stopwatchRef, defaultTimeUnit);
+  }
+
+  @Override
+  public void record(long amount, TimeUnit unit) {
+    if (amount >= 0) {
+      this.duration = this.duration.plus(amount, toChronoUnit(unit));

Review Comment:
   I feel like there might be some synchronization missing here. Duration is being reset potentially for multiple threads.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] nastra commented on pull request #5286: API: Introduce DefaultMetricsContext and Timer interface

Posted by GitBox <gi...@apache.org>.
nastra commented on PR #5286:
URL: https://github.com/apache/iceberg/pull/5286#issuecomment-1186967649

   @rdblue @danielcweeks I believe I addressed all comments, can you guys re-review please?


-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org