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

[ query ] executeStatement failed : Caused by: org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceFailureInfo$FailureException: null

    XMLWordPrintableJSON

Details

    • 2023-2-Query

    Description

      master 0215_077704e

      Query execution occasionally fails ,
      select min_time(s_0),max_time(s_0),count(s_0) from root.** align by device (1000dev * 2000 ts):
      2023-02-16 17:59:07,908 [pool-25-IoTDB-ClientRPC-Processor-121] INFO o.a.i.d.q.c.SessionManager:116 - IoTDB: Login status: Login successfully. User : root, opens Session-304-root:192.168.10.72:54342
      2023-02-16 17:59:07,982 [pool-25-IoTDB-ClientRPC-Processor-121] INFO o.a.i.d.s.t.i.ClientRPCServiceImpl:624 - IoTDB server version: 1.1.0-SNAPSHOT (Build: 077704e)
      2023-02-16 17:59:09,829 [Query-Worker-Thread-40$20230216_095908_30548_3.9.0.1] INFO o.a.i.d.e.c.TimeSeriesMetadataCache:86 - TimeseriesMetadataCache size = 16476198218
      2023-02-16 17:59:09,834 [Query-Worker-Thread-30$20230216_095908_30548_3.6.0.1] INFO o.a.i.d.e.c.BloomFilterCache:56 - BloomFilterCache size = 82380991
      2023-02-16 17:59:10,061 [Query-Worker-Thread-2$20230216_095908_30548_3.27.0.1] INFO o.a.i.d.e.c.ChunkCache:67 - ChunkCache size = 8238099109
      2023-02-16 17:59:13,827 [pool-25-IoTDB-ClientRPC-Processor-121] WARN o.a.i.d.u.ErrorHandlingUtils:89 - Status code: 301, Query Statement: "select min_time(s_0),max_time(s_0),count(s_0) from root.** align by device". executeStatement failed
      org.apache.iotdb.commons.exception.IoTDBException: org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceFailureInfo$FailureException

      at org.apache.iotdb.db.mpp.plan.execution.QueryExecution.dealWithException(QueryExecution.java:467)
      at org.apache.iotdb.db.mpp.plan.execution.QueryExecution.getResult(QueryExecution.java:450)
      at org.apache.iotdb.db.mpp.plan.execution.QueryExecution.getByteBufferBatchResult(QueryExecution.java:485)
      at org.apache.iotdb.db.utils.QueryDataSetUtils.convertQueryResultByFetchSize(QueryDataSetUtils.java:254)
      at org.apache.iotdb.db.service.thrift.impl.ClientRPCServiceImpl.lambda$static$0(ClientRPCServiceImpl.java:165)
      at org.apache.iotdb.db.service.thrift.impl.ClientRPCServiceImpl.executeStatementInternal(ClientRPCServiceImpl.java:239)
      at org.apache.iotdb.db.service.thrift.impl.ClientRPCServiceImpl.executeStatementV2(ClientRPCServiceImpl.java:476)
      at org.apache.iotdb.service.rpc.thrift.IClientRPCService$Processor$executeStatementV2.getResult(IClientRPCService.java:3629)
      at org.apache.iotdb.service.rpc.thrift.IClientRPCService$Processor$executeStatementV2.getResult(IClientRPCService.java:3609)
      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)
      Caused by: org.apache.iotdb.db.mpp.execution.fragment.FragmentInstanceFailureInfo$FailureException: null
      at java.base/java.util.ArrayList$Itr.checkForComodification(ArrayList.java:1043)
      at java.base/java.util.ArrayList$Itr.remove(ArrayList.java:1011)
      at org.apache.iotdb.db.mpp.execution.driver.DataDriver.addUsedFilesForQuery(DataDriver.java:168)
      at org.apache.iotdb.db.mpp.execution.driver.DataDriver.addUsedFilesForQuery(DataDriver.java:148)
      at org.apache.iotdb.db.mpp.execution.driver.DataDriver.initQueryDataSource(DataDriver.java:135)
      at org.apache.iotdb.db.mpp.execution.driver.DataDriver.initialize(DataDriver.java:101)
      at org.apache.iotdb.db.mpp.execution.driver.DataDriver.init(DataDriver.java:63)
      at org.apache.iotdb.db.mpp.execution.driver.Driver.lambda$processFor$1(Driver.java:139)
      at org.apache.iotdb.db.mpp.execution.driver.Driver.tryWithLock(Driver.java:299)
      at org.apache.iotdb.db.mpp.execution.driver.Driver.processFor(Driver.java:125)
      at org.apache.iotdb.db.mpp.execution.schedule.DriverTaskThread.execute(DriverTaskThread.java:69)
      at org.apache.iotdb.db.mpp.execution.schedule.AbstractDriverThread.run(AbstractDriverThread.java:73)
      2023-02-16 17:59:13,827 [pool-25-IoTDB-ClientRPC-Processor-121$20230216_095908_30548_3] INFO o.a.i.d.m.p.Coordinator:221 - Cost: 5833 ms, sql is select min_time(s_0),max_time(s_0),count(s_0) from root.** align by device

      – Test procedure
      1. 192.168.10.72/73/74
      confignode env:
      MAX_HEAP_SIZE="8G"

      datanode env:
      MAX_HEAP_SIZE="256G"
      MAX_DIRECT_MEMORY_SIZE="32G"

      COMMON properties:
      schema_replication_factor=3
      data_replication_factor=3
      max_waiting_time_when_insert_blocked=3600000
      query_timeout_threshold=3600000

      2. run benchmark

      3. execute query
      select min_time(s_0),max_time(s_0),count(s_0) from root.** align by device
      query occasionally fails.

      Attachments

        1. check_iotdb_5547.sh
          0.3 kB
          刘珍
        2. iotdb.sh
          0.2 kB
          刘珍
        3. iotdb_5547.conf
          14 kB
          刘珍

        Issue Links

          Activity

            People

              xiangweiwei Alima777
              刘珍 刘珍
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: