Details
-
Bug
-
Status: Resolved
-
Major
-
Resolution: Duplicate
-
1.3.0
-
None
-
None
-
Ozone version:
1.3.0 compiled through the master branch
Description
The following problems occur when using TPC-DS benchmark to write Ozone 1 TB data in textfile format,and the textfile format is then written to the ORC storage format, OM leader log as followings:
22/06/18 20:43:15 INFO WriterImpl: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450846/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 options: Compress: SNAPPY buffer: 262144 22/06/18 20:43:15 INFO PhysicalFsWriter: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450847/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 blockSize: 268435456 compression: Compress: SNAPPY buffer: 262144 22/06/18 20:43:15 INFO WriterImpl: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450847/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 options: Compress: SNAPPY buffer: 262144 22/06/18 20:43:15 INFO PhysicalFsWriter: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450848/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 blockSize: 268435456 compression: Compress: SNAPPY buffer: 262144 22/06/18 20:43:15 INFO WriterImpl: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450848/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 options: Compress: SNAPPY buffer: 262144 22/06/18 20:43:15 INFO PhysicalFsWriter: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450849/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 blockSize: 268435456 compression: Compress: SNAPPY buffer: 262144 22/06/18 20:43:15 INFO WriterImpl: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040046447265028599383617_0026_m_000005_4278/ss_sold_date_sk=2450849/part-00005-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 options: Compress: SNAPPY buffer: 262144 22/06/18 20:43:19 WARN XceiverClientRatis: 3 way commit failed on pipeline Pipeline[ Id: 9f950817-8a47-413f-b447-37aee0458b67, Nodes: f379fcb8-f407-452a-a5fe-18f7e6e7e236{ip: 10.2.1.13, host: tg-local-bdworker-3.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}defc9b44-2971-45bd-89b4-a2b5202e22b7{ip: 10.2.1.11, host: tg-local-bdworker-4.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}55da63a5-cf2d-44cd-bb31-92eff17a9c3b{ip: 10.2.1.6, host: tg-local-bdworker-1.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}, ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:55da63a5-cf2d-44cd-bb31-92eff17a9c3b, CreationTimestamp2022-06-18T15:36:00.113+08:00[Asia/Shanghai]] java.util.concurrent.ExecutionException: org.apache.ratis.protocol.exceptions.TimeoutIOException: Request #695 timeout 180s at java.base/java.util.concurrent.CompletableFuture.reportGet(Unknown Source) at java.base/java.util.concurrent.CompletableFuture.get(Unknown Source) at org.apache.hadoop.hdds.scm.XceiverClientRatis.watchForCommit(XceiverClientRatis.java:263) at org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchForCommit(CommitWatcher.java:199) at org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchOnLastIndex(CommitWatcher.java:166) at org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.sendWatchForCommit(RatisBlockOutputStream.java:101) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.watchForCommit(BlockOutputStream.java:373) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:529) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.close(BlockOutputStream.java:543) at org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.close(BlockOutputStreamEntry.java:130) at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleStreamAction(KeyOutputStream.java:503) at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:477) at org.apache.hadoop.ozone.client.io.KeyOutputStream.close(KeyOutputStream.java:530) at org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:56) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) at org.apache.orc.impl.PhysicalFsWriter.close(PhysicalFsWriter.java:485) at org.apache.orc.impl.WriterImpl.close(WriterImpl.java:731) at org.apache.orc.mapreduce.OrcMapreduceRecordWriter.close(OrcMapreduceRecordWriter.java:86) at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.close(OrcOutputWriter.scala:58) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.releaseCurrentWriter(FileFormatDataWriter.scala:64) at org.apache.spark.sql.execution.datasources.BaseDynamicPartitionDataWriter.renewCurrentWriter(FileFormatDataWriter.scala:266) at org.apache.spark.sql.execution.datasources.DynamicPartitionDataSingleWriter.write(FileFormatDataWriter.scala:357) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithMetrics(FileFormatDataWriter.scala:85) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithIterator(FileFormatDataWriter.scala:92) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:304) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1496) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:311) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$16(FileFormatWriter.scala:229) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) at java.base/java.lang.Thread.run(Unknown Source) Caused by: org.apache.ratis.protocol.exceptions.TimeoutIOException: Request #695 timeout 180s at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$timeoutCheck$5(GrpcClientProtocolClient.java:368) at java.base/java.util.Optional.ifPresent(Unknown Source) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.handleReplyFuture(GrpcClientProtocolClient.java:373) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.timeoutCheck(GrpcClientProtocolClient.java:368) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$onNext$1(GrpcClientProtocolClient.java:357) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:141) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:155) at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:38) at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:79) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) at java.base/java.util.concurrent.FutureTask.run(Unknown Source) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown Source) ... 3 more 22/06/18 20:43:19 INFO XceiverClientRatis: Could not commit index 15862 on pipeline Pipeline[ Id: 9f950817-8a47-413f-b447-37aee0458b67, Nodes: f379fcb8-f407-452a-a5fe-18f7e6e7e236{ip: 10.2.1.13, host: tg-local-bdworker-3.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}defc9b44-2971-45bd-89b4-a2b5202e22b7{ip: 10.2.1.11, host: tg-local-bdworker-4.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}55da63a5-cf2d-44cd-bb31-92eff17a9c3b{ip: 10.2.1.6, host: tg-local-bdworker-1.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}, ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:55da63a5-cf2d-44cd-bb31-92eff17a9c3b, CreationTimestamp2022-06-18T15:36:00.113+08:00[Asia/Shanghai]] to all the nodes. Server f379fcb8-f407-452a-a5fe-18f7e6e7e236 has failed. Committed by majority. 22/06/18 20:43:19 WARN BlockOutputStream: Failed to commit BlockId conID: 213 locID: 109611004723215417 bcsId: 15862 on Pipeline[ Id: 9f950817-8a47-413f-b447-37aee0458b67, Nodes: f379fcb8-f407-452a-a5fe-18f7e6e7e236{ip: 10.2.1.13, host: tg-local-bdworker-3.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}defc9b44-2971-45bd-89b4-a2b5202e22b7{ip: 10.2.1.11, host: tg-local-bdworker-4.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}55da63a5-cf2d-44cd-bb31-92eff17a9c3b{ip: 10.2.1.6, host: tg-local-bdworker-1.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}, ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:55da63a5-cf2d-44cd-bb31-92eff17a9c3b, CreationTimestamp2022-06-18T15:36:00.113+08:00[Asia/Shanghai]]. Failed nodes: [f379fcb8-f407-452a-a5fe-18f7e6e7e236{ip: null, host: null, ports: [], networkLocation: /default-rack, certSerialId: null, persistedOpState: null, persistedOpStateExpiryEpochSec: 0}] 22/06/18 20:43:19 INFO PhysicalFsWriter: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040049125352375803648532_0026_m_000020_4293/ss_sold_date_sk=2450827/part-00020-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 blockSize: 268435456 compression: Compress: SNAPPY buffer: 262144 22/06/18 20:43:19 INFO WriterImpl: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040049125352375803648532_0026_m_000020_4293/ss_sold_date_sk=2450827/part-00020-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 options: Compress: SNAPPY buffer: 262144 22/06/18 20:43:19 INFO PhysicalFsWriter: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040049125352375803648532_0026_m_000020_4293/ss_sold_date_sk=2450828/part-00020-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 blockSize: 268435456 compression: Compress: SNAPPY buffer: 262144 22/06/18 20:43:19 INFO WriterImpl: ORC writer created for path: ofs://cluster1/tgwarehouse/tpcdsorc.db/store_sales/.spark-staging-023388f9-3fa4-469f-b7bd-9f4688f52757/_temporary/0/_temporary/attempt_202206182040049125352375803648532_0026_m_000020_4293/ss_sold_date_sk=2450828/part-00020-023388f9-3fa4-469f-b7bd-9f4688f52757.c000.snappy.orc with stripeSize: 67108864 options: Compress: SNAPPY buffer: 262144 22/06/18 20:46:15 WARN XceiverClientRatis: 3 way commit failed on pipeline Pipeline[ Id: 50acd0d6-925e-48d2-986b-b66f9fd624a8, Nodes: 1b140ce9-fbac-4e2c-8adb-7486c67a200f{ip: 10.2.1.5, host: tg-local-bdworker-2.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}defc9b44-2971-45bd-89b4-a2b5202e22b7{ip: 10.2.1.11, host: tg-local-bdworker-4.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}f379fcb8-f407-452a-a5fe-18f7e6e7e236{ip: 10.2.1.13, host: tg-local-bdworker-3.tg.mt.com, ports: [REPLICATION=9886, RATIS=9858, RATIS_ADMIN=9857, RATIS_SERVER=9856, STANDALONE=9859], networkLocation: /default-rack, certSerialId: null, persistedOpState: IN_SERVICE, persistedOpStateExpiryEpochSec: 0}, ReplicationConfig: RATIS/THREE, State:OPEN, leaderId:1b140ce9-fbac-4e2c-8adb-7486c67a200f, CreationTimestamp2022-06-18T15:36:00.102+08:00[Asia/Shanghai]] java.util.concurrent.ExecutionException: org.apache.ratis.protocol.exceptions.TimeoutIOException: Request #797 timeout 180s at java.base/java.util.concurrent.CompletableFuture.reportGet(Unknown Source) at java.base/java.util.concurrent.CompletableFuture.get(Unknown Source) at org.apache.hadoop.hdds.scm.XceiverClientRatis.watchForCommit(XceiverClientRatis.java:263) at org.apache.hadoop.hdds.y.storage.CommitWatcher.watchForCommit(CommitWatcher.java:199) at org.apache.hadoop.hdds.scm.storage.CommitWatcher.watchOnLastIndex(CommitWatcher.java:166) at org.apache.hadoop.hdds.scm.storage.RatisBlockOutputStream.sendWatchForCommit(RatisBlockOutputStream.java:101) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.watchForCommit(BlockOutputStream.java:373) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.handleFlush(BlockOutputStream.java:529) at org.apache.hadoop.hdds.scm.storage.BlockOutputStream.close(BlockOutputStream.java:543) at org.apache.hadoop.ozone.client.io.BlockOutputStreamEntry.close(BlockOutputStreamEntry.java:130) at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleStreamAction(KeyOutputStream.java:503) at org.apache.hadoop.ozone.client.io.KeyOutputStream.handleFlushOrClose(KeyOutputStream.java:477) at org.apache.hadoop.ozone.client.io.KeyOutputStream.close(KeyOutputStream.java:530) at org.apache.hadoop.fs.ozone.OzoneFSOutputStream.close(OzoneFSOutputStream.java:56) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) at org.apache.orc.impl.PhysicalFsWriter.close(PhysicalFsWriter.java:485) at org.apache.orc.impl.WriterImpl.close(WriterImpl.java:731) at org.apache.orc.mapreduce.OrcMapreduceRecordWriter.close(OrcMapreduceRecordWriter.java:86) at org.apache.spark.sql.execution.datasources.orc.OrcOutputWriter.close(OrcOutputWriter.scala:58) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.releaseCurrentWriter(FileFormatDataWriter.scala:64) at org.apache.spark.sql.execution.datasources.BaseDynamicPartitionDataWriter.renewCurrentWriter(FileFormatDataWriter.scala:266) at org.apache.spark.sql.execution.datasources.DynamicPartitionDataSingleWriter.write(FileFormatDataWriter.scala:357) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithMetrics(FileFormatDataWriter.scala:85) at org.apache.spark.sql.execution.datasources.FileFormatDataWriter.writeWithIterator(FileFormatDataWriter.scala:92) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$executeTask$1(FileFormatWriter.scala:304) at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1496) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:311) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$16(FileFormatWriter.scala:229) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) at java.base/java.lang.Thread.run(Unknown Source) Caused by: org.apache.ratis.protocol.exceptions.TimeoutIOException: Request #797 timeout 180s at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$timeoutCheck$5(GrpcClientProtocolClient.java:368) at java.base/java.util.Optional.ifPresent(Unknown Source) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.handleReplyFuture(GrpcClientProtocolClient.java:373) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.timeoutCheck(GrpcClientProtocolClient.java:368) at org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.lambda$onNext$1(GrpcClientProtocolClient.java:357) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:141) at org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:155) at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:38) at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:79) at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) at java.base/java.util.concurrent.FutureTask.run(Unknown Source) at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown Source)
To solve the problem temporarily, use the following methods to close all pipelines and auto create them as follows:
ozone admin pipeline close $ALL_PIPELINES
Attachments
Attachments
Issue Links
- is duplicated by
-
HDDS-9323 Apply expiry of excluded datanodes to writing Ratis keys
- Resolved