[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-19 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16656477#comment-16656477
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol closed pull request #6814: [FLINK-10423][rocksdb][metrics] rocksdb 
native metrics monitor
URL: https://github.com/apache/flink/pull/6814
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/docs/_includes/generated/rocks_db_native_metric_configuration.html 
b/docs/_includes/generated/rocks_db_native_metric_configuration.html
new file mode 100644
index 000..78fc523b35b
--- /dev/null
+++ b/docs/_includes/generated/rocks_db_native_metric_configuration.html
@@ -0,0 +1,116 @@
+
+
+
+Key
+Default
+Description
+
+
+
+
+
state.backend.rocksdb.metrics.actual-delayed-write-rate
+false
+Monitor the current actual delayed write rate. 0 means no 
delay.
+
+
+state.backend.rocksdb.metrics.background-errors
+false
+Monitor the number of background errors in RocksDB.
+
+
+state.backend.rocksdb.metrics.compaction-pending
+false
+Track pending compactions in RocksDB. Returns 1 if a 
compaction is pending, 0 otherwise.
+
+
+
state.backend.rocksdb.metrics.cur-size-active-mem-table
+false
+Monitor the approximate size of the active memtable in 
bytes.
+
+
+
state.backend.rocksdb.metrics.cur-size-all-mem-tables
+false
+Monitor the approximate size of the active and unflushed 
immutable memtables in bytes.
+
+
+
state.backend.rocksdb.metrics.estimate-live-data-size
+false
+Estimate of the amount of live data in bytes.
+
+
+state.backend.rocksdb.metrics.estimate-num-keys
+false
+Estimate the number of keys in RocksDB.
+
+
+
state.backend.rocksdb.metrics.estimate-pending-compaction-bytes
+false
+Estimated total number of bytes compaction needs to rewrite to 
get all levels down to under target size. Not valid for other compactions than 
level-based.
+
+
+
state.backend.rocksdb.metrics.estimate-table-readers-mem
+false
+Estimate the memory used for reading SST tables, excluding 
memory used in block cache (e.g.,filter and index blocks) in bytes.
+
+
+
state.backend.rocksdb.metrics.mem-table-flush-pending
+false
+Monitor the number of pending memtable flushes in RocksDB.
+
+
+
state.backend.rocksdb.metrics.num-deletes-active-mem-table
+false
+Monitor the total number of delete entries in the active 
memtable.
+
+
+
state.backend.rocksdb.metrics.num-deletes-imm-mem-tables
+false
+Monitor the total number of delete entries in the unflushed 
immutable memtables.
+
+
+
state.backend.rocksdb.metrics.num-entries-active-mem-table
+false
+Monitor the total number of entries in the active 
memtable.
+
+
+
state.backend.rocksdb.metrics.num-entries-imm-mem-tables
+false
+Monitor the total number of entries in the unflushed immutable 
memtables.
+
+
+
state.backend.rocksdb.metrics.num-immutable-mem-table
+false
+Monitor the number of immutable memtables in RocksDB.
+
+
+state.backend.rocksdb.metrics.num-live-versions
+false
+Monitor number of live versions. Version is an internal data 
structure. See RocksDB file version_set.h for details. More live versions often 
mean more SST files are held from being deleted, by iterators or unfinished 
compactions.
+
+
+
state.backend.rocksdb.metrics.num-running-compactions
+false
+Monitor the number of currently running compactions.
+
+
+state.backend.rocksdb.metrics.num-running-flushes
+false
+Monitor the number of currently running flushes.
+
+
+state.backend.rocksdb.metrics.num-snapshots
+false
+Monitor the number of unreleased snapshots of the 
database.
+
+
+state.backend.rocksdb.metrics.size-all-mem-tables
+false
+Monitor

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-19 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16656474#comment-16656474
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r226577921
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,170 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   if (metricView.isClosed()) {
 
 Review comment:
   cool, thanks for looking into it. We can leave the code as is.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired he

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-17 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16653468#comment-16653468
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225909936
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,170 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   if (metricView.isClosed()) {
 
 Review comment:
   @zentol I did a little digging in the rocksdb code. The only place an 
exception that is explicitly thrown is in the JNI wrapper if the property 
string is invalid[1]. 
   
   Otherwise, the value is just pulled from a map here[2]. Since there is 
already a test validating each property against the current rocksdb version, 
and this pr only exposes an enum, I don't think there is any reason to believe 
a RocksDB exception will be thrown in production. 
   
   [1] 
https://github.com/facebook/rocksdb/blob/02dc07491650b0c8d8470a24806ba86d09ea31a4/java/rocksjni/rocksjni.cc#L1757
   
   [2] 
https://github.com/facebook/rocksdb/blob/189f0c27aaecdf17ae7fc1f826a423a28b77984f/db/internal_stats.cc#L339


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> --

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-17 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16653469#comment-16653469
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225909936
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,170 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   if (metricView.isClosed()) {
 
 Review comment:
   @zentol I did a little digging in the rocksdb code. The only place that an 
exception is explicitly thrown is in the JNI wrapper if the property string is 
invalid[1]. 
   
   Otherwise, the value is just pulled from a map here[2]. Since there is 
already a test validating each property against the current rocksdb version, 
and this pr only exposes an enum, I don't think there is any reason to believe 
a RocksDB exception will be thrown in production. 
   
   [1] 
https://github.com/facebook/rocksdb/blob/02dc07491650b0c8d8470a24806ba86d09ea31a4/java/rocksjni/rocksjni.cc#L1757
   
   [2] 
https://github.com/facebook/rocksdb/blob/189f0c27aaecdf17ae7fc1f826a423a28b77984f/db/internal_stats.cc#L339


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> --

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-17 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16653350#comment-16653350
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225872262
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,170 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   if (metricView.isClosed()) {
 
 Review comment:
   @StefanRRichter Do you know anything about the reliability of  
`RocksDB#getLongProperty`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doi

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-17 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16653320#comment-16653320
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225865415
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,170 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   if (metricView.isClosed()) {
 
 Review comment:
   ah. hmm...
   
   Let's say that an exception occurs when retrieving the value from RocksDB. 
How likely is it that this exception happens again? I.e. how reliable are 
failure-cases? Currently, if there is any intermittent failure the metric is 
disabled forever.
   
   If this is more of an all-or-nothing kind of thing we could try it out once 
before registering the metric and skipping the registration if it fails, and 
only skip repeated logging in the metric.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metr

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651361#comment-16651361
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225455306
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   private static final BigInteger ZERO = new BigInteger(1, new byte[]{0, 
0, 0, 0});
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //We use a local variable here to manually control the 
life-cycle.
+   // This allows us to verify that metrics do not try to access
+   // RocksDB after the monitor was closed.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
+   registry,
+   
UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(),
+   new OperatorID(),
+   OPERATOR_NAME
+   );
+
+   RocksDBNativeMetricOptions options = new 
RocksDBNativeMetricOptions();
+   // always returns a non-zero
+   // value since empty memtables
+   // have overhead.
+   options.enableSizeAllMemTables();
+
+   RocksDBNativeMetricMonitor monitor = new 
RocksDBNativeMetricMonitor(
+   rocksDBResource.getRocksDB(),
+   options,
+   group
+   );
+
+   ColumnFamilyHandle handle = 
rocksDBResource.createNewColumnFamily(COLUMN_FAMILY_NAME);
+   monitor.registerColumnFamily(COLUMN_FAMILY_NAME, handle);
+
+   Assert.assertEquals("Failed to register metrics for column 
family", 1, registry.metrics.size());
+
+   RocksDBNativeMetricMonitor.RocksDBNativeMetricView view = 
registry.metrics.get(0);
+
+   view.update();
+
+   Assert.assertNotEquals("Failed to pull metric from RocksDB", 
ZERO, view.getValue());
+
+   view.setValue(0L);
+
+   //After the monitor is closed no metric should be accessing 
RocksDB anymore.
+   //If they do, then this test will likely fail with a 
segmentation fault.
+   monitor.close();
+
+   rocksDBResource.after();
+
+   view.update();
+
+   Assert.assertEquals("Failed to release RocksDB reference", 
ZERO, view.getValue());
+   }
+
+   @Test
+   public void testReturnsUnsigned() throws Throwable {
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   Operato

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651363#comment-16651363
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225455353
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   private static final BigInteger ZERO = new BigInteger(1, new byte[]{0, 
0, 0, 0});
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //We use a local variable here to manually control the 
life-cycle.
+   // This allows us to verify that metrics do not try to access
+   // RocksDB after the monitor was closed.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
 
 Review comment:
   could be simplified by using a `GenericMetricGroup`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651358#comment-16651358
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225454760
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   private static final BigInteger ZERO = new BigInteger(1, new byte[]{0, 
0, 0, 0});
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //We use a local variable here to manually control the 
life-cycle.
+   // This allows us to verify that metrics do not try to access
+   // RocksDB after the monitor was closed.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
+   registry,
+   
UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(),
+   new OperatorID(),
+   OPERATOR_NAME
+   );
+
+   RocksDBNativeMetricOptions options = new 
RocksDBNativeMetricOptions();
+   // always returns a non-zero
+   // value since empty memtables
+   // have overhead.
+   options.enableSizeAllMemTables();
+
+   RocksDBNativeMetricMonitor monitor = new 
RocksDBNativeMetricMonitor(
+   rocksDBResource.getRocksDB(),
+   options,
+   group
+   );
+
+   ColumnFamilyHandle handle = 
rocksDBResource.createNewColumnFamily(COLUMN_FAMILY_NAME);
+   monitor.registerColumnFamily(COLUMN_FAMILY_NAME, handle);
+
+   Assert.assertEquals("Failed to register metrics for column 
family", 1, registry.metrics.size());
+
+   RocksDBNativeMetricMonitor.RocksDBNativeMetricView view = 
registry.metrics.get(0);
+
+   view.update();
+
+   Assert.assertNotEquals("Failed to pull metric from RocksDB", 
ZERO, view.getValue());
+
+   view.setValue(0L);
+
+   //After the monitor is closed no metric should be accessing 
RocksDB anymore.
+   //If they do, then this test will likely fail with a 
segmentation fault.
+   monitor.close();
+
+   rocksDBResource.after();
+
+   view.update();
+
+   Assert.assertEquals("Failed to release RocksDB reference", 
ZERO, view.getValue());
+   }
+
+   @Test
+   public void testReturnsUnsigned() throws Throwable {
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   Operato

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651359#comment-16651359
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225454717
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   private static final BigInteger ZERO = new BigInteger(1, new byte[]{0, 
0, 0, 0});
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //We use a local variable here to manually control the 
life-cycle.
+   // This allows us to verify that metrics do not try to access
+   // RocksDB after the monitor was closed.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
 
 Review comment:
   this can be simplified to `MetricGroup group = new 
UnregisteredMetricGroups()`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651354#comment-16651354
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225454449
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   rocksDB = null;
+   }
+   }
+
+   /**
+* A gauge which periodically pull a RocksDB native metric
 
 Review comment:
   pull -> pulls


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> ---

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651356#comment-16651356
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225454760
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   private static final BigInteger ZERO = new BigInteger(1, new byte[]{0, 
0, 0, 0});
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //We use a local variable here to manually control the 
life-cycle.
+   // This allows us to verify that metrics do not try to access
+   // RocksDB after the monitor was closed.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
+   registry,
+   
UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(),
+   new OperatorID(),
+   OPERATOR_NAME
+   );
+
+   RocksDBNativeMetricOptions options = new 
RocksDBNativeMetricOptions();
+   // always returns a non-zero
+   // value since empty memtables
+   // have overhead.
+   options.enableSizeAllMemTables();
+
+   RocksDBNativeMetricMonitor monitor = new 
RocksDBNativeMetricMonitor(
+   rocksDBResource.getRocksDB(),
+   options,
+   group
+   );
+
+   ColumnFamilyHandle handle = 
rocksDBResource.createNewColumnFamily(COLUMN_FAMILY_NAME);
+   monitor.registerColumnFamily(COLUMN_FAMILY_NAME, handle);
+
+   Assert.assertEquals("Failed to register metrics for column 
family", 1, registry.metrics.size());
+
+   RocksDBNativeMetricMonitor.RocksDBNativeMetricView view = 
registry.metrics.get(0);
+
+   view.update();
+
+   Assert.assertNotEquals("Failed to pull metric from RocksDB", 
ZERO, view.getValue());
+
+   view.setValue(0L);
+
+   //After the monitor is closed no metric should be accessing 
RocksDB anymore.
+   //If they do, then this test will likely fail with a 
segmentation fault.
+   monitor.close();
+
+   rocksDBResource.after();
+
+   view.update();
+
+   Assert.assertEquals("Failed to release RocksDB reference", 
ZERO, view.getValue());
+   }
+
+   @Test
+   public void testReturnsUnsigned() throws Throwable {
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   Operato

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651355#comment-16651355
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225454717
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,179 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.math.BigInteger;
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   private static final BigInteger ZERO = new BigInteger(1, new byte[]{0, 
0, 0, 0});
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //We use a local variable here to manually control the 
life-cycle.
+   // This allows us to verify that metrics do not try to access
+   // RocksDB after the monitor was closed.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
 
 Review comment:
   this can be simplified to `MetricGroup group = new 
UnregisteredMetricGroups()`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651352#comment-16651352
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225454242
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.Closeable;
+import java.math.BigInteger;
+
+/**
+ * A monitor which pulls {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   private final Object lock;
+
+   @GuardedBy("lock")
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
 
 Review comment:
   We've had issues in the past where the amount of logging messages where 
drowning out other messages.
   If this exception occurs, and if multiple metrics (or god forbid all of 
them) are enabled, then this could lead to a large amount of logging messages.
   
   I suggest to add a flag to each gauge so that this message is only logged 
once for each of them.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@i

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651346#comment-16651346
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225452792
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java
 ##
 @@ -192,7 +196,8 @@ public OperatorStateBackend createOperatorStateBackend(
streamOperator.getClass().getSimpleName(),
streamOperator,
typeSerializer,
-   closeableRegistry);
+   closeableRegistry,
+   
UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup());
 
 Review comment:
   use `new UnregisteredMetricsGroup()` instead. (separate class, more generic 
than the ones from `UnregisteredMetricGroups`. Also applies to other usages.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651344#comment-16651344
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225452289
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializer.java
 ##
 @@ -49,5 +51,7 @@ StreamOperatorStateContext streamOperatorStateContext(
@Nonnull String operatorClassName,
@Nonnull KeyContext keyContext,
@Nullable TypeSerializer keySerializer,
-   @Nonnull CloseableRegistry streamTaskCloseableRegistry) throws 
Exception;
+   @Nonnull CloseableRegistry streamTaskCloseableRegistry,
+   @Nonnull MetricGroup metricGroup
+   ) throws Exception;
 
 Review comment:
   indentation is off


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-16 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16651341#comment-16651341
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225451475
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
 ##
 @@ -456,7 +456,7 @@ public CheckpointStorage createCheckpointStorage(JobID 
jobId) throws IOException
priorityQueueStateType,
ttlTimeProvider,
getMemoryWatcherOptions(),
-   operatorMetricGroup
+   metricGroup
 
 Review comment:
   indentation is off


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650298#comment-16650298
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225188200
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
+   }
+   }
+
+   class RocksDBNativeMetricView implements Gauge, View {
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull ColumnFamilyHandle handle,
+   @Nonnull String property
+   ) {
+   this.handle = handle;
+   this.property = property;
+   }
+
+   

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650232#comment-16650232
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225171804
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
+   }
+   }
+
+   class RocksDBNativeMetricView implements Gauge, View {
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull ColumnFamilyHandle handle,
+   @Nonnull String property
+   ) {
+   this.handle = handle;
+   this.property = property;
+   }
+
+

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650230#comment-16650230
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225169917
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
 ##
 @@ -167,7 +206,9 @@
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
TaskKvStateRegistry kvStateRegistry,
-   TtlTimeProvider ttlTimeProvider) throws Exception;
+   TtlTimeProvider ttlTimeProvider,
+   OperatorMetricGroup operatorMetricGroup
 
 Review comment:
   remnant from an old attempt, reverting


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650226#comment-16650226
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225169192
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
+   }
+   }
+
+   class RocksDBNativeMetricView implements Gauge, View {
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull ColumnFamilyHandle handle,
+   @Nonnull String property
+   ) {
+   this.handle = handle;
+   this.property = property;
+   }
+
+

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650048#comment-16650048
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225113805
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
 ##
 @@ -405,14 +413,15 @@ public CheckpointStorage createCheckpointStorage(JobID 
jobId) throws IOException
 
@Override
public  AbstractKeyedStateBackend createKeyedStateBackend(
-   Environment env,
-   JobID jobID,
-   String operatorIdentifier,
-   TypeSerializer keySerializer,
-   int numberOfKeyGroups,
-   KeyGroupRange keyGroupRange,
-   TaskKvStateRegistry kvStateRegistry,
-   TtlTimeProvider ttlTimeProvider) throws IOException {
+   Environment env,
+   JobID jobID,
+   String operatorIdentifier,
 
 Review comment:
   revert indentation change


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650050#comment-16650050
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225112700
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/memory/MemoryStateBackend.java
 ##
 @@ -302,14 +303,15 @@ public OperatorStateBackend createOperatorStateBackend(
 
@Override
public  AbstractKeyedStateBackend createKeyedStateBackend(
-   Environment env,
-   JobID jobID,
-   String operatorIdentifier,
-   TypeSerializer keySerializer,
-   int numberOfKeyGroups,
-   KeyGroupRange keyGroupRange,
-   TaskKvStateRegistry kvStateRegistry,
-   TtlTimeProvider ttlTimeProvider) {
+   Environment env,
 
 Review comment:
   please revert the indentation change


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650049#comment-16650049
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225113222
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImplTest.java
 ##
 @@ -186,13 +191,15 @@ public OperatorStateBackend createOperatorStateBackend(
 
TypeSerializer typeSerializer = new IntSerializer();
CloseableRegistry closeableRegistry = new CloseableRegistry();
+   MetricGroup operatorMetricGroup = mock(MetricGroup.class);
 
 Review comment:
   unused?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650047#comment-16650047
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225112469
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
 ##
 @@ -167,7 +206,9 @@
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
TaskKvStateRegistry kvStateRegistry,
-   TtlTimeProvider ttlTimeProvider) throws Exception;
+   TtlTimeProvider ttlTimeProvider,
+   OperatorMetricGroup operatorMetricGroup
 
 Review comment:
   can this be a plain `MetricGroup`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650052#comment-16650052
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225116129
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,148 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //created as a local variable instead of using @Rule
+   //we we can manually control the lifecycle. Using @Rule
+   //but manually calling after causes occasional segmentation
 
 Review comment:
   This comment raises more questions that it answers.
   
   `We use a local variable here to manually control the life-cycle. This 
allows us to verify that metrics do not try to access RocksDB after the monitor 
was closed.`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650053#comment-16650053
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225117046
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitorTest.java
 ##
 @@ -0,0 +1,148 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MetricOptions;
+import org.apache.flink.metrics.Metric;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.metrics.MetricRegistry;
+import org.apache.flink.runtime.metrics.groups.AbstractMetricGroup;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.metrics.scope.ScopeFormats;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+
+/**
+ * validate native metric monitor.
+ */
+public class RocksDBNativeMetricMonitorTest {
+
+   private static final String OPERATOR_NAME = "dummy";
+
+   private static final String COLUMN_FAMILY_NAME = "column-family";
+
+   @Test
+   public void testMetricMonitorLifecycle() throws Throwable {
+   //created as a local variable instead of using @Rule
+   //we we can manually control the lifecycle. Using @Rule
+   //but manually calling after causes occasional segmentation
+   //faults.
+   RocksDBResource rocksDBResource = new RocksDBResource();
+   rocksDBResource.before();
+
+   SimpleMetricRegistry registry = new SimpleMetricRegistry();
+   OperatorMetricGroup group = new OperatorMetricGroup(
+   registry,
+   
UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(),
+   new OperatorID(),
+   OPERATOR_NAME
+   );
+
+   RocksDBNativeMetricOptions options = new 
RocksDBNativeMetricOptions();
+   // always returns a non-zero
+   // value since empty memtables
+   // have overhead.
+   options.enableSizeAllMemTables();
+
+   RocksDBNativeMetricMonitor monitor = new 
RocksDBNativeMetricMonitor(
+   rocksDBResource.getRocksDB(),
+   options,
+   group
+   );
+
+   ColumnFamilyHandle handle = 
rocksDBResource.createNewColumnFamily(COLUMN_FAMILY_NAME);
+   monitor.registerColumnFamily(COLUMN_FAMILY_NAME, handle);
+
+   Assert.assertEquals("Failed to register metrics for column 
family", 1, registry.metrics.size());
+
+   RocksDBNativeMetricMonitor.RocksDBNativeMetricView view = 
registry.metrics.get(0);
+
+   view.update();
+
+   Assert.assertNotEquals("Failed to pull metric from RocksDB", 0, 
Long.parseLong(view.getValue()));
+
+   view.setValue(0L);
+
+   //removing this line reliably causes segmentation faults
 
 Review comment:
   re-phrase: `After the monitor is closed no metric should be accessing 
RocksDB anymore. If they do, then this test will likely fail with a 
segmentation fault.`


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> ---

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650051#comment-16650051
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225114941
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
+   }
+   }
+
+   class RocksDBNativeMetricView implements Gauge, View {
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull ColumnFamilyHandle handle,
+   @Nonnull String property
+   ) {
+   this.handle = handle;
+   this.property = property;
+   }
+
+   

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16649997#comment-16649997
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225100731
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializer.java
 ##
 @@ -41,6 +42,7 @@
 * @param keyContext the key context of the operator instance for which 
the context is created Cannot be null.
 * @param keySerializer the key-serializer for the operator. Can be 
null.
 * @param streamTaskCloseableRegistry the closeable registry to which 
created closeable objects will be registered.
+* @param operatorMetricGroup TODO
 
 Review comment:
   Please complete the comment.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1664#comment-1664
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225103209
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
 
 Review comment:
   pull -> pulls


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650001#comment-16650001
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225104178
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
+   }
+   }
+
+   class RocksDBNativeMetricView implements Gauge, View {
 
 Review comment:
   I would suggest to add a short class-level documentation.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink me

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650003#comment-16650003
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225102395
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
+   }
+   }
+
+   class RocksDBNativeMetricView implements Gauge, View {
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+
 
 Review comment:
   2 newlines


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1665#comment-1665
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225103656
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
 
 Review comment:
   I would suggest a small comment about the relationship of `lock` and 
`rocksDB`, or maybe even better a `@GuardedBy("lock")` annotation.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16649998#comment-16649998
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225101104
 
 

 ##
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java
 ##
 @@ -159,6 +162,42 @@
 *
 * @throws Exception This method may forward all exceptions that occur 
while instantiating the backend.
 */
+   default  AbstractKeyedStateBackend createKeyedStateBackend(
+   Environment env,
+   JobID jobID,
+   String operatorIdentifier,
+   TypeSerializer keySerializer,
+   int numberOfKeyGroups,
+   KeyGroupRange keyGroupRange,
+   TaskKvStateRegistry kvStateRegistry,
+   TtlTimeProvider ttlTimeProvider
+   ) throws Exception {
+   return createKeyedStateBackend(
+   env,
+   jobID,
+   operatorIdentifier,
+   keySerializer,
+   numberOfKeyGroups,
+   keyGroupRange,
+   kvStateRegistry,
+   ttlTimeProvider,
+   
UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup()
+   );
+   }
+
+   /**
+* Creates a new {@link AbstractKeyedStateBackend} that is responsible 
for holding keyed state
+* and checkpointing it.
+*
+* Keyed State is state where each value is bound to a key.
+*
+* @param  The type of the keys by which the state is organized.
+*
+* @param operatorMetricGroup
 
 Review comment:
   Please complete comment.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-15 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16650002#comment-16650002
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r225102869
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,134 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.runtime.metrics.groups.OperatorMetricGroup;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricMonitor.class);
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final OperatorMetricGroup metricGroup;
+
+   private final Object lock;
+
+   private RocksDB rocksDB;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB rocksDB,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull OperatorMetricGroup metricGroup
+   ) {
+   this.options = options;
+   this.metricGroup = metricGroup;
+   this.rocksDB = rocksDB;
+
+   this.lock = new Object();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   MetricGroup group = metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(handle, property);
+   group.gauge(property, gauge);
+   }
+   }
+
+   /**
+* Updates the value of metricView if the reference is still valid.
+*/
+   private void setProperty(ColumnFamilyHandle handle, String property, 
RocksDBNativeMetricView metricView) {
+   try {
+   synchronized (lock) {
+   if (rocksDB != null) {
+   long value = 
rocksDB.getLongProperty(handle, property);
+   metricView.setValue(value);
+   }
+   }
+   } catch (RocksDBException e) {
+   LOG.warn("Failed to read native metric %s from 
RocksDB", property, e);
+   }
+   }
+
+   @Override
+   public void close() {
+   synchronized (lock) {
+   this.rocksDB = null;
 
 Review comment:
   for style consistency, `this.` is not required here.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Ke

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648269#comment-16648269
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224876736
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+   

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648194#comment-16648194
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224864142
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648180#comment-16648180
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224860246
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648178#comment-16648178
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224860246
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648174#comment-16648174
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224860246
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648176#comment-16648176
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224860246
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648152#comment-16648152
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224856170
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ##
 @@ -606,6 +627,17 @@ private RocksDB openDB(
Preconditions.checkState(1 + 
stateColumnFamilyDescriptors.size() == stateColumnFamilyHandles.size(),
"Not all requested column family handles have been 
created");
 
+   if (this.metricOptions.isEnabled()) {
+   this.nativeMetricMonitor = new 
RocksDBNativeMetricMonitor(
+   dbRef,
+   rocksDBResourceGuard,
+   metricOptions,
+   operatorMetricGroup
+   );
+
+   
this.cancelStreamRegistry.registerCloseable(nativeMetricMonitor);
 
 Review comment:
   It is not an issue in the sense of a bug, but it is much more explicit. 
`CloseableRegistry` is typically used to coordinate closing of resources across 
multiple threads and I would not use it if we move operate in just one thread.
   
   I think you need to close metrics as first thing in dispose() or else it 
will block. As a rule of thump, all closing in `dispose()`should happen in the 
reverse order of creation.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648147#comment-16648147
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224855111
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+   

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648146#comment-16648146
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224854199
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ##
 @@ -606,6 +627,17 @@ private RocksDB openDB(
Preconditions.checkState(1 + 
stateColumnFamilyDescriptors.size() == stateColumnFamilyHandles.size(),
"Not all requested column family handles have been 
created");
 
+   if (this.metricOptions.isEnabled()) {
+   this.nativeMetricMonitor = new 
RocksDBNativeMetricMonitor(
+   dbRef,
+   rocksDBResourceGuard,
+   metricOptions,
+   operatorMetricGroup
+   );
+
+   
this.cancelStreamRegistry.registerCloseable(nativeMetricMonitor);
 
 Review comment:
   Why is this an issue? The cancelStreamRegistry is closed in 
`AbstractKeyedStateBackend#dispose`.  I manually called close inside of 
`dispose` initially but I was concerned about the strict ordering it imposed. 
If the nativeMetricsWatcher wasn't closed first `rocksDBResourceGuard.close()` 
would block forever. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648142#comment-16648142
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224855415
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
 
 Review comment:
   Agreed, I am going to push the lease into the gauge itself. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648141#comment-16648141
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224855111
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+   

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16648136#comment-16648136
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224854199
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ##
 @@ -606,6 +627,17 @@ private RocksDB openDB(
Preconditions.checkState(1 + 
stateColumnFamilyDescriptors.size() == stateColumnFamilyHandles.size(),
"Not all requested column family handles have been 
created");
 
+   if (this.metricOptions.isEnabled()) {
+   this.nativeMetricMonitor = new 
RocksDBNativeMetricMonitor(
+   dbRef,
+   rocksDBResourceGuard,
+   metricOptions,
+   operatorMetricGroup
+   );
+
+   
this.cancelStreamRegistry.registerCloseable(nativeMetricMonitor);
 
 Review comment:
   Why is this an issue? The cancelStreamRegistry is closed in 
`AbstractKeyedStateBackend#dispose`.  I manually called close inside of 
`dispose` first but I was concerned about the strict ordering it imposed 
because if the nativeMetricsWatcher wasn't closed first 
`rocksDBResourceGuard.close()` would block forever. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16647998#comment-16647998
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224817432
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16647990#comment-16647990
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224761096
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-12 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16647828#comment-16647828
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224761096
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646800#comment-16646800
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224531500
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646802#comment-16646802
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224529147
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ##
 @@ -606,6 +627,17 @@ private RocksDB openDB(
Preconditions.checkState(1 + 
stateColumnFamilyDescriptors.size() == stateColumnFamilyHandles.size(),
"Not all requested column family handles have been 
created");
 
+   if (this.metricOptions.isEnabled()) {
+   this.nativeMetricMonitor = new 
RocksDBNativeMetricMonitor(
+   dbRef,
+   rocksDBResourceGuard,
+   metricOptions,
+   operatorMetricGroup
+   );
+
+   
this.cancelStreamRegistry.registerCloseable(nativeMetricMonitor);
 
 Review comment:
   I think this is not a good way for a clean shutdown. Instead I would suggest 
to close this in the `dispose()` method.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646801#comment-16646801
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224534588
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
+
+   private final RocksDBNativeMetricOptions options;
+
+   private final MetricGroup metricGroup;
+
+   RocksDBNativeMetricMonitor(
+   @Nonnull RocksDB db,
+   @Nonnull ResourceGuard guard,
+   @Nonnull RocksDBNativeMetricOptions options,
+   @Nonnull MetricGroup metricGroup
+   ) throws IOException {
+   this.db = db;
+   this.lease = guard.acquireResource();
+   this.options = options;
+   this.metricGroup = metricGroup;
+
+   this.registeredGauges = new CloseableRegistry();
+   }
+
+   /**
+* Register gauges to pull native metrics for the column family.
+* @param columnFamilyName group name for the new gauges
+* @param handle native handle to the column family
+*/
+   void registerColumnFamily(String columnFamilyName, ColumnFamilyHandle 
handle) {
+   try {
+   MetricGroup group = 
metricGroup.addGroup(columnFamilyName);
+
+   for (String property : options.getProperties()) {
+   RocksDBNativeMetricView gauge = new 
RocksDBNativeMetricView(
+   property,
+   handle,
+   db
+   );
+
+   group.gauge(property, gauge);
+   registeredGauges.registerCloseable(gauge);
+   }
+   } catch (IOException e) {
+   throw new FlinkRuntimeException("Unable to register 
native metrics with RocksDB", e);
+   }
+   }
+
+   @Override
+   public void close() {
+   IOUtils.closeQuietly(registeredGauges);
+   IOUtils.closeQuietly(lease);
+   }
+
+   static class RocksDBNativeMetricView implements Gauge, View, 
Closeable {
+   private static final Logger LOG = 
LoggerFactory.getLogger(RocksDBNativeMetricView.class);
+
+   private final String property;
+
+   private final ColumnFamilyHandle handle;
+
+   private final RocksDB db;
+
+   private volatile boolean open;
+
+   private long value;
+
+   private RocksDBNativeMetricView(
+   @Nonnull String property,
+  

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646803#comment-16646803
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224533004
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.fs.CloseableRegistry;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.View;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final CloseableRegistry registeredGauges;
+
+   private final RocksDB db;
+
+   private final ResourceGuard.Lease lease;
 
 Review comment:
   I think a lease is not required if this object is closed in the beginning 
`RocksDBKeyedStateBackend.dispose()`. The lease is rather required for parallel 
threads that use the db and we don't know when they are done using the db in 
`dispose()`.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-11 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16646804#comment-16646804
 ] 

ASF GitHub Bot commented on FLINK-10423:


StefanRRichter commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r224536940
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
 ##
 @@ -1400,7 +1433,8 @@ private ColumnFamilyHandle createColumnFamily(String 
stateName) {
ColumnFamilyDescriptor columnDescriptor = new 
ColumnFamilyDescriptor(nameBytes, columnOptions);
 
try {
-   return db.createColumnFamily(columnDescriptor);
+   ColumnFamilyHandle handle = 
db.createColumnFamily(columnDescriptor);
+   return handle;
 
 Review comment:
   Unrelated and could be reverted.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-10 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16645307#comment-16645307
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on issue #6814: [FLINK-10423][rocksdb][metrics] rocksdb 
native metrics monitor
URL: https://github.com/apache/flink/pull/6814#issuecomment-428659805
 
 
   @zentol I've updated the pr based on your comments. Thanks for pointing out 
`view`, that greatly simplified the implementation. 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-09 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16644320#comment-16644320
 ] 

ASF GitHub Bot commented on FLINK-10423:


sjwiesman commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r223912317
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
 ##
 @@ -250,7 +251,10 @@ public void testCorrectMergeOperatorSet() throws 
IOException {
enableIncrementalCheckpointing,
TestLocalRecoveryConfig.disabled(),
RocksDBStateBackend.PriorityQueueStateType.HEAP,
-   TtlTimeProvider.DEFAULT);
+   TtlTimeProvider.DEFAULT,
+   new RocksDBNativeMetricOptions(),
+   Optional.empty()
 
 Review comment:
   It is optional because I needed a default value for overriding the previous 
version of `StateBackend#createKeyedStateBackend`[1]. I was not aware of 
`UnregisteredOperatorMetricGroup` but that would make a better fallback, will 
update. 
   
   [1] 
https://github.com/apache/flink/blob/8859febcb3cbb592d7a80d5609f9b26925dc2d45/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateBackend.java#L165


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-09 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16644138#comment-16644138
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r223872647
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricTest.java
 ##
 @@ -0,0 +1,160 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.ResourceGuard;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests that native metrics can be pulled from RocksDB.
+ */
+public class RocksDBNativeMetricTest {
+
+   private static final String COLUMN_FAMILY_NAME = "test-cf";
+
+   private static final String PROPERTY = "property";
+
+   @Rule public RocksDBResource rocksDBResource = new RocksDBResource();
+
+   @Test
+   public void testMessageCollector() throws IOException {
+   ConcurrentLinkedQueue> 
messageQueue = new ConcurrentLinkedQueue<>();
+
+   RocksDBNativeMetricMonitor.RocksDBNativeMetricTask task = new 
RocksDBNativeMetricMonitor.RocksDBNativeMetricTask(
+   mock(RocksDB.class),
+   new ResourceGuard(),
+   new ArrayList<>(),
+   mock(MetricGroup.class),
+   messageQueue
+   );
+
+   messageQueue.add(Tuple2.of("", mock(ColumnFamilyHandle.class)));
+   task.collectNewMessages();
+
+   Assert.assertEquals("failed to removed pending messages from 
queue", 0, messageQueue.size());
+   Assert.assertEquals("failed to add message to kv state", 1, 
task.numberOfColumnFamilies());
+   }
+
+   @Test
+   public void testGaugeRegistration() throws IOException {
+   MetricGroup metricGroup = mock(MetricGroup.class);
+   MetricGroup innerGroup = mock(MetricGroup.class);
 
 Review comment:
   there are various utility classes to not require mocking for metric group 
related stuff. `UnregisteredMetricGroups` can be used to create safe default 
implementations where specific methods can be overridden.
   You can pass a custom MetricRegistry to check for specific metrics instead.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb

[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-09 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16644136#comment-16644136
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r223873851
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricOptions.java
 ##
 @@ -0,0 +1,278 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * Enable which RocksDB metrics to forward to Flink's metrics reporter.
+ * All metrics report at the column family level and return unsigned long 
values.
+ *
+ * Properties and doc comments are taken from RocksDB documentation. See
+ * https://github.com/facebook/rocksdb/blob/64324e329eb0a9b4e77241a425a1615ff524c7f1/include/rocksdb/db.h#L429";>
+ * db.h for more information.
+ */
+public class RocksDBNativeMetricOptions {
+
+   private static final long TEN_SECONDS = 10 * 1000;
+
+   private Set properties;
+
+   private long frequency = TEN_SECONDS;
+
+   public RocksDBNativeMetricOptions() {
+   this.properties = new HashSet<>();
+   }
+
+   /**
+* Returns number of immutable memtables that have not yet been flushed.
+*/
+   public void enableNumImmutableMemTable() {
+   this.properties.add("rocksdb.num-immutable-mem-table");
 
 Review comment:
   all of this must be configurable via the `flink-conf.yaml` configuration.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-09 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16644137#comment-16644137
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r223872847
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendTest.java
 ##
 @@ -250,7 +251,10 @@ public void testCorrectMergeOperatorSet() throws 
IOException {
enableIncrementalCheckpointing,
TestLocalRecoveryConfig.disabled(),
RocksDBStateBackend.PriorityQueueStateType.HEAP,
-   TtlTimeProvider.DEFAULT);
+   TtlTimeProvider.DEFAULT,
+   new RocksDBNativeMetricOptions(),
+   Optional.empty()
 
 Review comment:
   If this argument an optional purely to simplify tests? If so, please just 
use an `UnregisteredOperatorMetricGroup` instead.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (FLINK-10423) Forward RocksDB native metrics to Flink metrics reporter

2018-10-09 Thread ASF GitHub Bot (JIRA)


[ 
https://issues.apache.org/jira/browse/FLINK-10423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16644139#comment-16644139
 ] 

ASF GitHub Bot commented on FLINK-10423:


zentol commented on a change in pull request #6814: 
[FLINK-10423][rocksdb][metrics] rocksdb native metrics monitor
URL: https://github.com/apache/flink/pull/6814#discussion_r223873328
 
 

 ##
 File path: 
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBNativeMetricMonitor.java
 ##
 @@ -0,0 +1,195 @@
+/*
+ * 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.flink.contrib.streaming.state;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.util.IOUtils;
+import org.apache.flink.util.ResourceGuard;
+
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nonnull;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+/**
+ * A monitor which pull {{@link RocksDB}} native metrics
+ * and forwards them to Flink's metric group. All metrics are
+ * unsigned longs and are reported at the column family level.
+ */
+@Internal
+public class RocksDBNativeMetricMonitor implements Closeable {
+
+   private final TimerTask task;
+
+   private final Timer timer;
+
+   private final ConcurrentLinkedQueue> 
messageQueue;
+
+   RocksDBNativeMetricMonitor(
 
 Review comment:
   metric polling can be implemented using a `View`, which is periodically 
called to update the current value. This should simplify this class quite a bit.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Forward RocksDB native metrics to Flink metrics reporter 
> -
>
> Key: FLINK-10423
> URL: https://issues.apache.org/jira/browse/FLINK-10423
> Project: Flink
>  Issue Type: New Feature
>  Components: Metrics, State Backends, Checkpointing
>Reporter: Seth Wiesman
>Assignee: Seth Wiesman
>Priority: Major
>  Labels: pull-request-available
>
> RocksDB contains a number of metrics at the column family level about current 
> memory usage, open memtables,  etc that would be useful to users wishing 
> greater insight what rocksdb is doing. This work is inspired heavily by the 
> comments on this rocksdb issue thread 
> (https://github.com/facebook/rocksdb/issues/3216#issuecomment-348779233)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)