Uploaded image for project: 'Apache Hudi'
  1. Apache Hudi
  2. HUDI-4923

CI test flaky: TestHoodieReadClient.testReadFilterExistAfterBulkInsertPrepped

    XMLWordPrintableJSON

Details

    Description

      TestHoodieReadClient.testReadFilterExistAfterBulkInsertPrepped is flaky in CI. once in 2 to 3 runs, it fails. 

       

      https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_apis/build/builds/11744/logs/42

       

      2022-09-26T19:23:05.8173562Z [ERROR] Tests run: 9, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 117.898 s <<< FAILURE! - in org.apache.hudi.client.TestHoodieReadClient
      2022-09-26T19:23:05.8210175Z [ERROR] testReadFilterExistAfterBulkInsertPrepped  Time elapsed: 6.887 s  <<< ERROR!
      2022-09-26T19:23:05.8211083Z org.apache.spark.SparkException: 
      2022-09-26T19:23:05.8212525Z Job aborted due to stage failure: Task 1 in stage 40.0 failed 1 times, most recent failure: Lost task 1.0 in stage 40.0 (TID 40, localhost, executor driver): org.apache.hudi.exception.HoodieMetadataException: Failed to retrieve files in partition /tmp/junit5315216008976338235/dataset/2015/03/16 from metadata
      2022-09-26T19:23:05.8213799Z 	at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:143)
      2022-09-26T19:23:05.8214617Z 	at org.apache.hudi.metadata.HoodieMetadataFileSystemView.listPartition(HoodieMetadataFileSystemView.java:65)
      2022-09-26T19:23:05.8215554Z 	at org.apache.hudi.common.table.view.AbstractTableFileSystemView.lambda$ensurePartitionLoadedCorrectly$9(AbstractTableFileSystemView.java:306)
      2022-09-26T19:23:05.8216431Z 	at java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1660)
      2022-09-26T19:23:05.8217264Z 	at org.apache.hudi.common.table.view.AbstractTableFileSystemView.ensurePartitionLoadedCorrectly(AbstractTableFileSystemView.java:297)
      2022-09-26T19:23:05.8218524Z 	at org.apache.hudi.common.table.view.AbstractTableFileSystemView.getLatestBaseFilesBeforeOrOn(AbstractTableFileSystemView.java:518)
      2022-09-26T19:23:05.8219891Z 	at org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForPartition(HoodieIndexUtils.java:70)
      2022-09-26T19:23:05.8220664Z 	at org.apache.hudi.index.HoodieIndexUtils.lambda$getLatestBaseFilesForAllPartitions$ff6885d8$1(HoodieIndexUtils.java:110)
      2022-09-26T19:23:05.8222270Z 	at org.apache.hudi.client.common.HoodieSparkEngineContext.lambda$flatMap$7d470b86$1(HoodieSparkEngineContext.java:137)
      2022-09-26T19:23:05.8223164Z 	at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$1$1.apply(JavaRDDLike.scala:125)
      2022-09-26T19:23:05.8223881Z 	at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$1$1.apply(JavaRDDLike.scala:125)
      2022-09-26T19:23:05.8224772Z 	at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:435)
      2022-09-26T19:23:05.8225418Z 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441)
      2022-09-26T19:23:05.8226172Z 	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
      2022-09-26T19:23:05.8226731Z 	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
      2022-09-26T19:23:05.8227332Z 	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
      2022-09-26T19:23:05.8228154Z 	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
      2022-09-26T19:23:05.8228794Z 	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
      2022-09-26T19:23:05.8229438Z 	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
      2022-09-26T19:23:05.8230041Z 	at scala.collection.AbstractIterator.to(Iterator.scala:1334)
      2022-09-26T19:23:05.8230637Z 	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
      2022-09-26T19:23:05.8231275Z 	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1334)
      2022-09-26T19:23:05.8231907Z 	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
      2022-09-26T19:23:05.8232674Z 	at scala.collection.AbstractIterator.toArray(Iterator.scala:1334)
      2022-09-26T19:23:05.8233483Z 	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:945)
      2022-09-26T19:23:05.8234163Z 	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:945)
      2022-09-26T19:23:05.8234828Z 	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
      2022-09-26T19:23:05.8235518Z 	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
      2022-09-26T19:23:05.8236173Z 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      2022-09-26T19:23:05.8236743Z 	at org.apache.spark.scheduler.Task.run(Task.scala:123)
      2022-09-26T19:23:05.8237367Z 	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
      2022-09-26T19:23:05.8238040Z 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
      2022-09-26T19:23:05.8238651Z 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
      2022-09-26T19:23:05.8239314Z 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      2022-09-26T19:23:05.8240013Z 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      2022-09-26T19:23:05.8240575Z 	at java.lang.Thread.run(Thread.java:750)
      2022-09-26T19:23:05.8241234Z Caused by: org.apache.hudi.exception.HoodieIOException: Could not check if /tmp/junit5315216008976338235/dataset/.hoodie/metadata is a valid table
      2022-09-26T19:23:05.8242067Z 	at org.apache.hudi.exception.TableNotFoundException.checkTableValidity(TableNotFoundException.java:59)
      2022-09-26T19:23:05.8242822Z 	at org.apache.hudi.common.table.HoodieTableMetaClient.<init>(HoodieTableMetaClient.java:128)
      2022-09-26T19:23:05.8243588Z 	at org.apache.hudi.common.table.HoodieTableMetaClient.newMetaClient(HoodieTableMetaClient.java:642)
      2022-09-26T19:23:05.8244425Z 	at org.apache.hudi.common.table.HoodieTableMetaClient.access$000(HoodieTableMetaClient.java:80)
      2022-09-26T19:23:05.8245193Z 	at org.apache.hudi.common.table.HoodieTableMetaClient$Builder.build(HoodieTableMetaClient.java:711)
      2022-09-26T19:23:05.8246144Z 	at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.<init>(AbstractHoodieLogRecordReader.java:160)
      2022-09-26T19:23:05.8246989Z 	at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:92)
      2022-09-26T19:23:05.8247843Z 	at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.<init>(HoodieMetadataMergedLogRecordReader.java:63)
      2022-09-26T19:23:05.8248696Z 	at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.<init>(HoodieMetadataMergedLogRecordReader.java:51)
      2022-09-26T19:23:05.8249589Z 	at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader$Builder.build(HoodieMetadataMergedLogRecordReader.java:230)
      2022-09-26T19:23:05.8250679Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getLogRecordScanner(HoodieBackedTableMetadata.java:507)
      2022-09-26T19:23:05.8251467Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.openReaders(HoodieBackedTableMetadata.java:420)
      2022-09-26T19:23:05.8252447Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getOrCreateReaders(HoodieBackedTableMetadata.java:405)
      2022-09-26T19:23:05.8253304Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$getRecordsByKeys$1(HoodieBackedTableMetadata.java:212)
      2022-09-26T19:23:05.8253981Z 	at java.util.HashMap.forEach(HashMap.java:1290)
      2022-09-26T19:23:05.8254635Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:210)
      2022-09-26T19:23:05.8255452Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:142)
      2022-09-26T19:23:05.8256234Z 	at org.apache.hudi.metadata.BaseTableMetadata.fetchAllFilesInPartition(BaseTableMetadata.java:323)
      2022-09-26T19:23:05.8257010Z 	at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:141)
      2022-09-26T19:23:05.8287990Z 	... 35 more
      2022-09-26T19:23:05.8289671Z Caused by: java.net.ConnectException: Call From fv-az94-675/10.1.0.27 to localhost:34997 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
      2022-09-26T19:23:05.8290670Z 	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
      2022-09-26T19:23:05.8291340Z 	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
      2022-09-26T19:23:05.8292140Z 	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
      2022-09-26T19:23:05.8292838Z 	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
      2022-09-26T19:23:05.8293449Z 	at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:827)
      2022-09-26T19:23:05.8294068Z 	at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:757)
      2022-09-26T19:23:05.8294653Z 	at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1553)
      2022-09-26T19:23:05.8295218Z 	at org.apache.hadoop.ipc.Client.call(Client.java:1495)
      2022-09-26T19:23:05.8295761Z 	at org.apache.hadoop.ipc.Client.call(Client.java:1394)
      2022-09-26T19:23:05.8296364Z 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
      2022-09-26T19:23:05.8297075Z 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
      2022-09-26T19:23:05.8297976Z 	at com.sun.proxy.$Proxy86.getFileInfo(Unknown Source)
      2022-09-26T19:23:05.8298866Z 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:800)
      2022-09-26T19:23:05.8299590Z 	at sun.reflect.GeneratedMethodAccessor178.invoke(Unknown Source)
      2022-09-26T19:23:05.8300191Z 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      2022-09-26T19:23:05.8300981Z 	at java.lang.reflect.Method.invoke(Method.java:498)
      2022-09-26T19:23:05.8301626Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:422)
      2022-09-26T19:23:05.8302419Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
      2022-09-26T19:23:05.8303479Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
      2022-09-26T19:23:05.8304245Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
      2022-09-26T19:23:05.8305152Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
      2022-09-26T19:23:05.8305929Z 	at com.sun.proxy.$Proxy87.getFileInfo(Unknown Source)
      2022-09-26T19:23:05.8306617Z 	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1673)
      2022-09-26T19:23:05.8307269Z 	at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1524)
      2022-09-26T19:23:05.8308248Z 	at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1521)
      2022-09-26T19:23:05.8308978Z 	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
      2022-09-26T19:23:05.8309719Z 	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1521)
      2022-09-26T19:23:05.8310517Z 	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$getFileStatus$17(HoodieWrapperFileSystem.java:402)
      2022-09-26T19:23:05.8311568Z 	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:106)
      2022-09-26T19:23:05.8312398Z 	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.getFileStatus(HoodieWrapperFileSystem.java:396)
      2022-09-26T19:23:05.8313180Z 	at org.apache.hudi.exception.TableNotFoundException.checkTableValidity(TableNotFoundException.java:51)
      2022-09-26T19:23:05.8313738Z 	... 53 more
      2022-09-26T19:23:05.8314118Z Caused by: java.net.ConnectException: Connection refused
      2022-09-26T19:23:05.8314599Z 	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
      2022-09-26T19:23:05.8315172Z 	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:716)
      2022-09-26T19:23:05.8315977Z 	at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
      2022-09-26T19:23:05.8316567Z 	at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:532)
      2022-09-26T19:23:05.8317787Z 	at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:701)
      2022-09-26T19:23:05.8318493Z 	at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:814)
      2022-09-26T19:23:05.8319148Z 	at org.apache.hadoop.ipc.Client$Connection.access$3700(Client.java:423)
      2022-09-26T19:23:05.8319767Z 	at org.apache.hadoop.ipc.Client.getConnection(Client.java:1610)
      2022-09-26T19:23:05.8320313Z 	at org.apache.hadoop.ipc.Client.call(Client.java:1441)
      2022-09-26T19:23:05.8320731Z 	... 76 more
      2022-09-26T19:23:05.8320858Z 
      2022-09-26T19:23:05.8321134Z Driver stacktrace:
      2022-09-26T19:23:05.8321790Z 	at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1889)
      2022-09-26T19:23:05.8323200Z 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1877)
      2022-09-26T19:23:05.8324382Z 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1876)
      2022-09-26T19:23:05.8325116Z 	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
      2022-09-26T19:23:05.8325757Z 	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
      2022-09-26T19:23:05.8326400Z 	at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1876)
      2022-09-26T19:23:05.8327275Z 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:926)
      2022-09-26T19:23:05.8328219Z 	at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:926)
      2022-09-26T19:23:05.8328868Z 	at scala.Option.foreach(Option.scala:257)
      2022-09-26T19:23:05.8329455Z 	at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:926)
      2022-09-26T19:23:05.8330178Z 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2110)
      2022-09-26T19:23:05.8331138Z 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2059)
      2022-09-26T19:23:05.8331886Z 	at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2048)
      2022-09-26T19:23:05.8332554Z 	at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
      2022-09-26T19:23:05.8333178Z 	at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:737)
      2022-09-26T19:23:05.8333795Z 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
      2022-09-26T19:23:05.8334372Z 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2082)
      2022-09-26T19:23:05.8334960Z 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2101)
      2022-09-26T19:23:05.8335639Z 	at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126)
      2022-09-26T19:23:05.8336469Z 	at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945)
      2022-09-26T19:23:05.8337122Z 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
      2022-09-26T19:23:05.8458886Z 	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
      2022-09-26T19:23:05.8459543Z 	at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
      2022-09-26T19:23:05.8460073Z 	at org.apache.spark.rdd.RDD.collect(RDD.scala:944)
      2022-09-26T19:23:05.8460660Z 	at org.apache.spark.api.java.JavaRDDLike$class.collect(JavaRDDLike.scala:361)
      2022-09-26T19:23:05.8461464Z 	at org.apache.spark.api.java.AbstractJavaRDDLike.collect(JavaRDDLike.scala:45)
      2022-09-26T19:23:05.8462366Z 	at org.apache.hudi.client.common.HoodieSparkEngineContext.flatMap(HoodieSparkEngineContext.java:137)
      2022-09-26T19:23:05.8463202Z 	at org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForAllPartitions(HoodieIndexUtils.java:108)
      2022-09-26T19:23:05.8463989Z 	at org.apache.hudi.index.bloom.HoodieBloomIndex.loadColumnRangesFromFiles(HoodieBloomIndex.java:164)
      2022-09-26T19:23:05.8464801Z 	at org.apache.hudi.index.bloom.HoodieBloomIndex.getBloomIndexFileInfoForPartitions(HoodieBloomIndex.java:149)
      2022-09-26T19:23:05.8465597Z 	at org.apache.hudi.index.bloom.HoodieBloomIndex.lookupIndex(HoodieBloomIndex.java:123)
      2022-09-26T19:23:05.8466296Z 	at org.apache.hudi.index.bloom.HoodieBloomIndex.tagLocation(HoodieBloomIndex.java:89)
      2022-09-26T19:23:05.8467069Z 	at org.apache.hudi.client.SparkRDDReadClient.tagLocation(SparkRDDReadClient.java:218)
      2022-09-26T19:23:05.8467782Z 	at org.apache.hudi.client.SparkRDDReadClient.filterExists(SparkRDDReadClient.java:205)
      2022-09-26T19:23:05.8468512Z 	at org.apache.hudi.client.TestHoodieReadClient.testReadFilterExist(TestHoodieReadClient.java:123)
      2022-09-26T19:23:05.8469352Z 	at org.apache.hudi.client.TestHoodieReadClient.testReadFilterExistAfterBulkInsertPrepped(TestHoodieReadClient.java:80)
      2022-09-26T19:23:05.8470205Z 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      2022-09-26T19:23:05.8470762Z 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      2022-09-26T19:23:05.8471734Z 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      2022-09-26T19:23:05.8472298Z 	at java.lang.reflect.Method.invoke(Method.java:498)
      2022-09-26T19:23:05.8472838Z 	at org.junit.platform.commons.util.ReflectionUtils.invokeMethod(ReflectionUtils.java:688)
      2022-09-26T19:23:05.8473675Z 	at org.junit.jupiter.engine.execution.MethodInvocation.proceed(MethodInvocation.java:60)
      2022-09-26T19:23:05.8475448Z 	at org.junit.jupiter.engine.execution.InvocationInterceptorChain$ValidatingInvocation.proceed(InvocationInterceptorChain.java:131)
      2022-09-26T19:23:05.8476336Z 	at org.junit.jupiter.engine.extension.TimeoutExtension.intercept(TimeoutExtension.java:149)
      2022-09-26T19:23:05.8477112Z 	at org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestableMethod(TimeoutExtension.java:140)
      2022-09-26T19:23:05.8477920Z 	at org.junit.jupiter.engine.extension.TimeoutExtension.interceptTestMethod(TimeoutExtension.java:84)
      2022-09-26T19:23:05.8479206Z 	at org.junit.jupiter.engine.execution.ExecutableInvoker$ReflectiveInterceptorCall.lambda$ofVoidMethod$0(ExecutableInvoker.java:115)
      2022-09-26T19:23:05.8506134Z 	at org.junit.jupiter.engine.execution.ExecutableInvoker.lambda$invoke$0(ExecutableInvoker.java:105)
      2022-09-26T19:23:05.8506939Z 	at org.junit.jupiter.engine.execution.InvocationInterceptorChain$InterceptedInvocation.proceed(InvocationInterceptorChain.java:106)
      2022-09-26T19:23:05.8507724Z 	at org.junit.jupiter.engine.execution.InvocationInterceptorChain.proceed(InvocationInterceptorChain.java:64)
      2022-09-26T19:23:05.8508671Z 	at org.junit.jupiter.engine.execution.InvocationInterceptorChain.chainAndInvoke(InvocationInterceptorChain.java:45)
      2022-09-26T19:23:05.8509692Z 	at org.junit.jupiter.engine.execution.InvocationInterceptorChain.invoke(InvocationInterceptorChain.java:37)
      2022-09-26T19:23:05.8510458Z 	at org.junit.jupiter.engine.execution.ExecutableInvoker.invoke(ExecutableInvoker.java:104)
      2022-09-26T19:23:05.8511123Z 	at org.junit.jupiter.engine.execution.ExecutableInvoker.invoke(ExecutableInvoker.java:98)
      2022-09-26T19:23:05.8512567Z 	at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.lambda$invokeTestMethod$6(TestMethodTestDescriptor.java:210)
      2022-09-26T19:23:05.8513406Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8514800Z 	at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.invokeTestMethod(TestMethodTestDescriptor.java:206)
      2022-09-26T19:23:05.8515873Z 	at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:131)
      2022-09-26T19:23:05.8516985Z 	at org.junit.jupiter.engine.descriptor.TestMethodTestDescriptor.execute(TestMethodTestDescriptor.java:65)
      2022-09-26T19:23:05.8517833Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$5(NodeTestTask.java:139)
      2022-09-26T19:23:05.8518655Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8519498Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$7(NodeTestTask.java:129)
      2022-09-26T19:23:05.8520258Z 	at org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
      2022-09-26T19:23:05.8521321Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:127)
      2022-09-26T19:23:05.8522158Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8522968Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:126)
      2022-09-26T19:23:05.8523737Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:84)
      2022-09-26T19:23:05.8524377Z 	at java.util.ArrayList.forEach(ArrayList.java:1259)
      2022-09-26T19:23:05.8526190Z 	at org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.invokeAll(SameThreadHierarchicalTestExecutorService.java:38)
      2022-09-26T19:23:05.8527236Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$5(NodeTestTask.java:143)
      2022-09-26T19:23:05.8528078Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8529536Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$7(NodeTestTask.java:129)
      2022-09-26T19:23:05.8530325Z 	at org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
      2022-09-26T19:23:05.8531085Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:127)
      2022-09-26T19:23:05.8532085Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8533175Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:126)
      2022-09-26T19:23:05.8534383Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:84)
      2022-09-26T19:23:05.8535428Z 	at java.util.ArrayList.forEach(ArrayList.java:1259)
      2022-09-26T19:23:05.8536219Z 	at org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.invokeAll(SameThreadHierarchicalTestExecutorService.java:38)
      2022-09-26T19:23:05.8537177Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$5(NodeTestTask.java:143)
      2022-09-26T19:23:05.8538320Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8539156Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$7(NodeTestTask.java:129)
      2022-09-26T19:23:05.8540331Z 	at org.junit.platform.engine.support.hierarchical.Node.around(Node.java:137)
      2022-09-26T19:23:05.8541403Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.lambda$executeRecursively$8(NodeTestTask.java:127)
      2022-09-26T19:23:05.8542186Z 	at org.junit.platform.engine.support.hierarchical.ThrowableCollector.execute(ThrowableCollector.java:73)
      2022-09-26T19:23:05.8543311Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.executeRecursively(NodeTestTask.java:126)
      2022-09-26T19:23:05.8544061Z 	at org.junit.platform.engine.support.hierarchical.NodeTestTask.execute(NodeTestTask.java:84)
      2022-09-26T19:23:05.8544927Z 	at org.junit.platform.engine.support.hierarchical.SameThreadHierarchicalTestExecutorService.submit(SameThreadHierarchicalTestExecutorService.java:32)
      2022-09-26T19:23:05.8546191Z 	at org.junit.platform.engine.support.hierarchical.HierarchicalTestExecutor.execute(HierarchicalTestExecutor.java:57)
      2022-09-26T19:23:05.8547001Z 	at org.junit.platform.engine.support.hierarchical.HierarchicalTestEngine.execute(HierarchicalTestEngine.java:51)
      2022-09-26T19:23:05.8548117Z 	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:108)
      2022-09-26T19:23:05.8549276Z 	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:88)
      2022-09-26T19:23:05.8550113Z 	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.lambda$execute$0(EngineExecutionOrchestrator.java:54)
      2022-09-26T19:23:05.8551012Z 	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.withInterceptedStreams(EngineExecutionOrchestrator.java:67)
      2022-09-26T19:23:05.8551884Z 	at org.junit.platform.launcher.core.EngineExecutionOrchestrator.execute(EngineExecutionOrchestrator.java:52)
      2022-09-26T19:23:05.8552786Z 	at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:96)
      2022-09-26T19:23:05.8553790Z 	at org.junit.platform.launcher.core.DefaultLauncher.execute(DefaultLauncher.java:75)
      2022-09-26T19:23:05.8554523Z 	at org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invokeAllTests(JUnitPlatformProvider.java:150)
      2022-09-26T19:23:05.8555539Z 	at org.apache.maven.surefire.junitplatform.JUnitPlatformProvider.invoke(JUnitPlatformProvider.java:124)
      2022-09-26T19:23:05.8556353Z 	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:384)
      2022-09-26T19:23:05.8557129Z 	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:345)
      2022-09-26T19:23:05.8557818Z 	at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:126)
      2022-09-26T19:23:05.8558795Z 	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:418)
      2022-09-26T19:23:05.8559690Z Caused by: org.apache.hudi.exception.HoodieMetadataException: Failed to retrieve files in partition /tmp/junit5315216008976338235/dataset/2015/03/16 from metadata
      2022-09-26T19:23:05.8560804Z 	at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:143)
      2022-09-26T19:23:05.8561740Z 	at org.apache.hudi.metadata.HoodieMetadataFileSystemView.listPartition(HoodieMetadataFileSystemView.java:65)
      2022-09-26T19:23:05.8562678Z 	at org.apache.hudi.common.table.view.AbstractTableFileSystemView.lambda$ensurePartitionLoadedCorrectly$9(AbstractTableFileSystemView.java:306)
      2022-09-26T19:23:05.8563677Z 	at java.util.concurrent.ConcurrentHashMap.computeIfAbsent(ConcurrentHashMap.java:1660)
      2022-09-26T19:23:05.8564827Z 	at org.apache.hudi.common.table.view.AbstractTableFileSystemView.ensurePartitionLoadedCorrectly(AbstractTableFileSystemView.java:297)
      2022-09-26T19:23:05.8565864Z 	at org.apache.hudi.common.table.view.AbstractTableFileSystemView.getLatestBaseFilesBeforeOrOn(AbstractTableFileSystemView.java:518)
      2022-09-26T19:23:05.8567212Z 	at org.apache.hudi.index.HoodieIndexUtils.getLatestBaseFilesForPartition(HoodieIndexUtils.java:70)
      2022-09-26T19:23:05.8568477Z 	at org.apache.hudi.index.HoodieIndexUtils.lambda$getLatestBaseFilesForAllPartitions$ff6885d8$1(HoodieIndexUtils.java:110)
      2022-09-26T19:23:05.8569383Z 	at org.apache.hudi.client.common.HoodieSparkEngineContext.lambda$flatMap$7d470b86$1(HoodieSparkEngineContext.java:137)
      2022-09-26T19:23:05.8570195Z 	at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$1$1.apply(JavaRDDLike.scala:125)
      2022-09-26T19:23:05.8571165Z 	at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$1$1.apply(JavaRDDLike.scala:125)
      2022-09-26T19:23:05.8571966Z 	at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:435)
      2022-09-26T19:23:05.8573060Z 	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:441)
      2022-09-26T19:23:05.8573592Z 	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
      2022-09-26T19:23:05.8574114Z 	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
      2022-09-26T19:23:05.8574695Z 	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
      2022-09-26T19:23:05.8575306Z 	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
      2022-09-26T19:23:05.8575902Z 	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
      2022-09-26T19:23:05.8576677Z 	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
      2022-09-26T19:23:05.8587879Z 	at scala.collection.AbstractIterator.to(Iterator.scala:1334)
      2022-09-26T19:23:05.8588742Z 	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
      2022-09-26T19:23:05.8589359Z 	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1334)
      2022-09-26T19:23:05.8590099Z 	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
      2022-09-26T19:23:05.8590651Z 	at scala.collection.AbstractIterator.toArray(Iterator.scala:1334)
      2022-09-26T19:23:05.8591224Z 	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:945)
      2022-09-26T19:23:05.8591983Z 	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:945)
      2022-09-26T19:23:05.8592662Z 	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
      2022-09-26T19:23:05.8593407Z 	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
      2022-09-26T19:23:05.8593957Z 	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
      2022-09-26T19:23:05.8594439Z 	at org.apache.spark.scheduler.Task.run(Task.scala:123)
      2022-09-26T19:23:05.8594962Z 	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
      2022-09-26T19:23:05.8595524Z 	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
      2022-09-26T19:23:05.8596037Z 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
      2022-09-26T19:23:05.8596597Z 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
      2022-09-26T19:23:05.8597358Z 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
      2022-09-26T19:23:05.8597845Z 	at java.lang.Thread.run(Thread.java:750)
      2022-09-26T19:23:05.8598424Z Caused by: org.apache.hudi.exception.HoodieIOException: Could not check if /tmp/junit5315216008976338235/dataset/.hoodie/metadata is a valid table
      2022-09-26T19:23:05.8599157Z 	at org.apache.hudi.exception.TableNotFoundException.checkTableValidity(TableNotFoundException.java:59)
      2022-09-26T19:23:05.8600043Z 	at org.apache.hudi.common.table.HoodieTableMetaClient.<init>(HoodieTableMetaClient.java:128)
      2022-09-26T19:23:05.8600864Z 	at org.apache.hudi.common.table.HoodieTableMetaClient.newMetaClient(HoodieTableMetaClient.java:642)
      2022-09-26T19:23:05.8601518Z 	at org.apache.hudi.common.table.HoodieTableMetaClient.access$000(HoodieTableMetaClient.java:80)
      2022-09-26T19:23:05.8602154Z 	at org.apache.hudi.common.table.HoodieTableMetaClient$Builder.build(HoodieTableMetaClient.java:711)
      2022-09-26T19:23:05.8602842Z 	at org.apache.hudi.common.table.log.AbstractHoodieLogRecordReader.<init>(AbstractHoodieLogRecordReader.java:160)
      2022-09-26T19:23:05.8603554Z 	at org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner.<init>(HoodieMergedLogRecordScanner.java:92)
      2022-09-26T19:23:05.8604360Z 	at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.<init>(HoodieMetadataMergedLogRecordReader.java:63)
      2022-09-26T19:23:05.8605094Z 	at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader.<init>(HoodieMetadataMergedLogRecordReader.java:51)
      2022-09-26T19:23:05.8606032Z 	at org.apache.hudi.metadata.HoodieMetadataMergedLogRecordReader$Builder.build(HoodieMetadataMergedLogRecordReader.java:230)
      2022-09-26T19:23:05.8606797Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getLogRecordScanner(HoodieBackedTableMetadata.java:507)
      2022-09-26T19:23:05.8607488Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.openReaders(HoodieBackedTableMetadata.java:420)
      2022-09-26T19:23:05.8608195Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getOrCreateReaders(HoodieBackedTableMetadata.java:405)
      2022-09-26T19:23:05.8608936Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.lambda$getRecordsByKeys$1(HoodieBackedTableMetadata.java:212)
      2022-09-26T19:23:05.8609685Z 	at java.util.HashMap.forEach(HashMap.java:1290)
      2022-09-26T19:23:05.8610239Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordsByKeys(HoodieBackedTableMetadata.java:210)
      2022-09-26T19:23:05.8611121Z 	at org.apache.hudi.metadata.HoodieBackedTableMetadata.getRecordByKey(HoodieBackedTableMetadata.java:142)
      2022-09-26T19:23:05.8611826Z 	at org.apache.hudi.metadata.BaseTableMetadata.fetchAllFilesInPartition(BaseTableMetadata.java:323)
      2022-09-26T19:23:05.8612740Z 	at org.apache.hudi.metadata.BaseTableMetadata.getAllFilesInPartition(BaseTableMetadata.java:141)
      2022-09-26T19:23:05.8613278Z 	... 35 more
      2022-09-26T19:23:05.8614904Z Caused by: java.net.ConnectException: Call From fv-az94-675/10.1.0.27 to localhost:34997 failed on connection exception: java.net.ConnectException: Connection refused; For more details see:  http://wiki.apache.org/hadoop/ConnectionRefused
      2022-09-26T19:23:05.8616170Z 	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
      2022-09-26T19:23:05.8616945Z 	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
      2022-09-26T19:23:05.8618518Z 	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
      2022-09-26T19:23:05.8619234Z 	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
      2022-09-26T19:23:05.8619846Z 	at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:827)
      2022-09-26T19:23:05.8620453Z 	at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:757)
      2022-09-26T19:23:05.8621033Z 	at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1553)
      2022-09-26T19:23:05.8621601Z 	at org.apache.hadoop.ipc.Client.call(Client.java:1495)
      2022-09-26T19:23:05.8622137Z 	at org.apache.hadoop.ipc.Client.call(Client.java:1394)
      2022-09-26T19:23:05.8622897Z 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
      2022-09-26T19:23:05.8623766Z 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
      2022-09-26T19:23:05.8624370Z 	at com.sun.proxy.$Proxy86.getFileInfo(Unknown Source)
      2022-09-26T19:23:05.8625061Z 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:800)
      2022-09-26T19:23:05.8626517Z 	at sun.reflect.GeneratedMethodAccessor178.invoke(Unknown Source)
      2022-09-26T19:23:05.8627157Z 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      2022-09-26T19:23:05.8627772Z 	at java.lang.reflect.Method.invoke(Method.java:498)
      2022-09-26T19:23:05.8628417Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:422)
      2022-09-26T19:23:05.8629545Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
      2022-09-26T19:23:05.8630454Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
      2022-09-26T19:23:05.8631831Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
      2022-09-26T19:23:05.8632791Z 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
      2022-09-26T19:23:05.8633410Z 	at com.sun.proxy.$Proxy87.getFileInfo(Unknown Source)
      2022-09-26T19:23:05.8633948Z 	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1673)
      2022-09-26T19:23:05.8634623Z 	at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1524)
      2022-09-26T19:23:05.8635355Z 	at org.apache.hadoop.hdfs.DistributedFileSystem$29.doCall(DistributedFileSystem.java:1521)
      2022-09-26T19:23:05.8636564Z 	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
      2022-09-26T19:23:05.8637333Z 	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1521)
      2022-09-26T19:23:05.8638130Z 	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$getFileStatus$17(HoodieWrapperFileSystem.java:402)
      2022-09-26T19:23:05.8638980Z 	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:106)
      2022-09-26T19:23:05.8639808Z 	at org.apache.hudi.common.fs.HoodieWrapperFileSystem.getFileStatus(HoodieWrapperFileSystem.java:396)
      2022-09-26T19:23:05.8640612Z 	at org.apache.hudi.exception.TableNotFoundException.checkTableValidity(TableNotFoundException.java:51)
      2022-09-26T19:23:05.8641150Z 	... 53 more
      2022-09-26T19:23:05.8641533Z Caused by: java.net.ConnectException: Connection refused
      2022-09-26T19:23:05.8642177Z 	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
      2022-09-26T19:23:05.8642876Z 	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:716)
      2022-09-26T19:23:05.8643717Z 	at org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:206)
      2022-09-26T19:23:05.8644351Z 	at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:532)
      2022-09-26T19:23:05.8644945Z 	at org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:701)
      2022-09-26T19:23:05.8645599Z 	at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:814)
      2022-09-26T19:23:05.8646241Z 	at org.apache.hadoop.ipc.Client$Connection.access$3700(Client.java:423)
      2022-09-26T19:23:05.8646988Z 	at org.apache.hadoop.ipc.Client.getConnection(Client.java:1610)
      2022-09-26T19:23:05.8647538Z 	at org.apache.hadoop.ipc.Client.call(Client.java:1441)
      2022-09-26T19:23:05.8647923Z 	... 76 more 

      Attachments

        Issue Links

          Activity

            People

              shivnarayan sivabalan narayanan
              shivnarayan sivabalan narayanan
              Raymond Xu
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: