bookkeeper
bookkeeper copied to clipboard
Bookie could coredump when it is shutting down
BUG REPORT
Describe the bug
When we kill the bookie processor and shut it down, sometimes core dump could happen. Core dump log:
--------------- T H R E A D ---------------
Current thread (0x00007f14a7128350): JavaThread "qtp1834755909-1155" [_thread_in_native, id=7358, stack(0x00007f053cbfc000,0x00007f053ccfd000)]
Stack: [0x00007f053cbfc000,0x00007f053ccfd000], sp=0x00007f053ccfb130, free space=1020k
Native frames: (J=compiled Java code, j=interpreted, Vv=VM code, C=native code)
C [librocksdbjni3874548073448578178.so+0x37b79b] rocksdb::DBImpl::GetIntProperty(rocksdb::ColumnFamilyHandle*, rocksdb::Slice const&, unsigned long*)+0x2b
C [librocksdbjni3874548073448578178.so+0x291bb1] Java_org_rocksdb_RocksDB_getLongProperty+0x171
J 7450 org.rocksdb.RocksDB.getLongProperty(JJLjava/lang/String;I)J (0 bytes) @ 0x00007f1495901a0d [0x00007f1495901920+0x00000000000000ed]
J 9392 c2 org.apache.bookkeeper.bookie.storage.ldb.EntryLocationIndexStats$1.getSample()Ljava/lang/Number; (5 bytes) @ 0x00007f1495344cac [0x00007f1495344be0+0x00000000000000cc]
J 6119 c2 org.apache.bookkeeper.stats.prometheus.SimpleGauge.getSample()Ljava/lang/Number; (10 bytes) @ 0x00007f1495829f84 [0x00007f1495829f20+0x0000000000000064]
J 5356 c2 org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider$$Lambda$310+0x0000000800fa6460.accept(Ljava/lang/Object;Ljava/lang/Object;)V (20 bytes) @ 0x00007f14957245f4 [0x0000
7f14957234a0+0x0000000000001154]
J 6418 c2 java.util.concurrent.ConcurrentHashMap.forEach(Ljava/util/function/BiConsumer;)V [email protected] (65 bytes) @ 0x00007f149554f0d8 [0x00007f149554ee40+0x0000000000000298]
J 10308 c2 org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider.writeAllMetrics(Ljava/io/Writer;)V (65 bytes) @ 0x00007f149592c3b0 [0x00007f149592c1a0+0x0000000000000210]
j org.apache.bookkeeper.stats.prometheus.PrometheusServlet.doGet(Ljavax/servlet/http/HttpServletRequest;Ljavax/servlet/http/HttpServletResponse;)V+29
To Reproduce
It's not easy to reprodce the core dump everytime. Howerver we could find that it is related to rocksdb and PrometheusServlet from the core dump log:
- The PrometheusServlet still services when bookie shutting down half
- At this time a
GET
request for Prometheus will callrocksdb::DBImpl::GetIntProperty
, while maybe the rocksdb has been closed now. Then core dump happends
So the root casue could be calling rocksdb#getLongProperty
after rocksdb has been closed
And we could reproduce the rocksdb core dump from the following code:
<dependency>
<groupId>org.apache.bookkeeper</groupId>
<artifactId>bookkeeper-server</artifactId>
<version>4.15.4</version>
</dependency>
public static void main(String[] args) throws Exception {
RocksDB.loadLibrary();
try (final ColumnFamilyOptions cfOpts = new ColumnFamilyOptions().optimizeUniversalStyleCompaction()) {
final List<ColumnFamilyDescriptor> cfDescriptors = Arrays.asList(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, cfOpts),
new ColumnFamilyDescriptor("test-cf".getBytes(), cfOpts));
final List<ColumnFamilyHandle> columnFamilyHandleList = new ArrayList<>();
try (final DBOptions options = new DBOptions().setCreateIfMissing(true).setCreateMissingColumnFamilies(true);
final RocksDB db = RocksDB.open(options, "/Users/houxiaoyu/test/testrocksdb", cfDescriptors, columnFamilyHandleList)) {
try {
db.put("key2".getBytes(), "value".getBytes());
} finally {
db.close();
// calling `rocksdb#getLongProperty` after rocksdb has been closed
db.getLongProperty("rocksdb.estimate-num-keys");
}
}
}
}