Uploaded image for project: 'Apache IoTDB'
  1. Apache IoTDB
  2. IOTDB-5664

[compaction] java.lang.OutOfMemoryError: Java heap space

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Blocker
    • Resolution: Unresolved
    • 1.1.0-SNAPSHOT
    • None
    • None
    • 2023-3-Storage

    Description

      rc/1.1.0 0306_32f9db0
      启动1C1D,写入1个顺序tsfile,其他tsfile为乱序,每个序列(普通/对齐)写入1个点,flush一次。
      46小时,OOM
      2023-03-12 14:53:49,951 [pool-24-IoTDB-DataNodeInternalRPC-Processor-24] ERROR o.a.i.c.c.t.WrappedThreadPoolExecutor:108 - Exception in thread pool org.apache.iotdb.threadpool:type=DataNodeInternalRPC-Processor
      java.lang.OutOfMemoryError: Java heap space: failed reallocation of scalar replaced objects
      2023-03-12 14:57:58,102 [pool-7-IoTDB-StorageEngine-1415] ERROR o.a.i.c.c.t.WrappedThreadPoolExecutor:108 - Exception in thread pool org.apache.iotdb.threadpool:type=StorageEngine
      java.lang.OutOfMemoryError: Java heap space
      2023-03-12 14:58:03,694 [pool-24-IoTDB-DataNodeInternalRPC-Processor-24] ERROR o.a.i.c.c.IoTDBDefaultThreadExceptionHandler:31 - Exception in thread pool-24-IoTDB-DataNodeInternalRPC-Processor-24-20462
      java.lang.OutOfMemoryError: Java heap space: failed reallocation of scalar replaced objects
      2023-03-12 14:53:49,951 [pool-24-IoTDB-DataNodeInternalRPC-Processor-17] ERROR o.a.i.c.c.t.WrappedThreadPoolExecutor:108 - Exception in thread pool org.apache.iotdb.threadpool:type=DataNodeInternalRPC-Processor
      java.lang.OutOfMemoryError: Java heap space
      2023-03-12 14:53:49,951 [pool-21-IoTDB-MPPCoordinator-1] ERROR o.a.i.c.c.t.WrappedThreadPoolExecutor:108 - Exception in thread pool org.apache.iotdb.threadpool:type=MPPCoordinator
      java.lang.OutOfMemoryError: Java heap space
      2023-03-12 14:58:03,694 [pool-21-IoTDB-MPPCoordinator-2] ERROR o.a.i.c.c.t.WrappedThreadPoolExecutor:108 - Exception in thread pool org.apache.iotdb.threadpool:type=MPPCoordinator
      java.lang.OutOfMemoryError: Java heap space
      2023-03-12 14:54:01,138 [pool-21-IoTDB-MPPCoordinator-4] ERROR o.a.i.c.c.IoTDBDefaultThreadExceptionHandler:31 - Exception in thread pool-21-IoTDB-MPPCoordinator-4-62
      java.lang.OutOfMemoryError: Java heap space
      2023-03-12 14:57:52,533 [pool-24-IoTDB-DataNodeInternalRPC-Processor-77] ERROR o.a.i.c.c.t.WrappedThreadPoolExecutor:108 - Exception in thread pool org.apache.iotdb.threadpool:type=DataNodeInternalRPC-Processor
      java.lang.OutOfMemoryError: Java heap space
      2023-03-12 14:54:12,287 [pool-25-IoTDB-ClientRPC-Processor-22385$20230312_064221_26096_1.1.0] ERROR o.a.i.d.m.e.e.RegionWriteExecutor:92 - Java heap space
      java.lang.OutOfMemoryError: Java heap space
      at java.base/java.lang.StringLatin1.toLowerCase(StringLatin1.java:397)
      at java.base/java.lang.String.toLowerCase(String.java:2503)
      at java.base/java.lang.String.toLowerCase(String.java:2527)
      at org.apache.iotdb.commons.service.metric.enums.Metric.toString(Metric.java:96)
      at org.apache.iotdb.db.mpp.metric.PerformanceOverviewMetricsManager.recordScheduleSchemaValidateCost(PerformanceOverviewMetricsManager.java:88)
      at org.apache.iotdb.db.mpp.execution.executor.RegionWriteExecutor$WritePlanNodeExecutionVisitor.executeDataInsert(RegionWriteExecutor.java:228)
      at org.apache.iotdb.db.mpp.execution.executor.RegionWriteExecutor$WritePlanNodeExecutionVisitor.visitInsertTablet(RegionWriteExecutor.java:185)
      at org.apache.iotdb.db.mpp.execution.executor.RegionWriteExecutor$WritePlanNodeExecutionVisitor.visitInsertTablet(RegionWriteExecutor.java:139)
      at org.apache.iotdb.db.mpp.plan.planner.plan.node.write.InsertTabletNode.accept(InsertTabletNode.java:1062)
      at org.apache.iotdb.db.mpp.execution.executor.RegionWriteExecutor.execute(RegionWriteExecutor.java:90)
      at org.apache.iotdb.db.mpp.plan.scheduler.FragmentInstanceDispatcherImpl.dispatchLocally(FragmentInstanceDispatcherImpl.java:321)
      at org.apache.iotdb.db.mpp.plan.scheduler.FragmentInstanceDispatcherImpl.dispatchOneInstance(FragmentInstanceDispatcherImpl.java:211)
      at org.apache.iotdb.db.mpp.plan.scheduler.FragmentInstanceDispatcherImpl.dispatchWriteAsync(FragmentInstanceDispatcherImpl.java:182)
      at org.apache.iotdb.db.mpp.plan.scheduler.FragmentInstanceDispatcherImpl.dispatch(FragmentInstanceDispatcherImpl.java:101)
      at org.apache.iotdb.db.mpp.plan.scheduler.ClusterScheduler.start(ClusterScheduler.java:116)
      at org.apache.iotdb.db.mpp.plan.execution.QueryExecution.schedule(QueryExecution.java:312)
      at org.apache.iotdb.db.mpp.plan.execution.QueryExecution.start(QueryExecution.java:215)
      at org.apache.iotdb.db.mpp.plan.Coordinator.execute(Coordinator.java:161)
      at org.apache.iotdb.db.mpp.plan.Coordinator.execute(Coordinator.java:175)
      at org.apache.iotdb.db.service.thrift.impl.ClientRPCServiceImpl.insertTablet(ClientRPCServiceImpl.java:1378)
      at org.apache.iotdb.service.rpc.thrift.IClientRPCService$Processor$insertTablet.getResult(IClientRPCService.java:4279)
      at org.apache.iotdb.service.rpc.thrift.IClientRPCService$Processor$insertTablet.getResult(IClientRPCService.java:4259)
      at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:38)
      at org.apache.iotdb.db.service.thrift.ProcessorWithMetrics.process(ProcessorWithMetrics.java:64)
      at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:248)
      at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
      at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
      at java.base/java.lang.Thread.run(Thread.java:834)

      1.测试环境
      私有云1期 172.16.2.15
      CN ENV:
      MAX_HEAP_SIZE="2G"
      cn_target_config_node_list=172.16.2.15:10710
      DN ENV:
      MAX_HEAP_SIZE="20G"
      MAX_DIRECT_MEMORY_SIZE="6G"

      2. 启动运行脚本
      bm配置文件有2个,见附件
      在benchmark dir下运行run_comp.sh

      Attachments

        1. image-2023-03-23-11-18-00-061.png
          1.25 MB
          周沛辰
        2. run_comp.sh
          2 kB
          刘珍
        3. test.sh
          0.2 kB
          刘珍
        4. comp1_normal.conf
          14 kB
          刘珍
        5. comp3_aligned.conf
          14 kB
          刘珍

        Activity

          People

            ChouBenson 周沛辰
            刘珍 刘珍
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: