trino
trino copied to clipboard
java.lang.RuntimeException: org.apache.kudu.client.NonRecoverableException: Scanner c1042093aae341e9bc1808ae78804e91 not found (it may have expired)
java.lang.RuntimeException: org.apache.kudu.client.NonRecoverableException: Scanner c1042093aae341e9bc1808ae78804e91 not found (it may have expired)
Could you share the Trino version, steps to reproduce and the full stacktrace?
@ebyhr trino368
Can you test with 371? It includes a client bump and fix for some bug in Kudu: https://github.com/trinodb/trino/pull/10940
We are using Trino 386, and encounter the same problem. The failure information is:
{
"type": "java.lang.RuntimeException",
"message": "org.apache.kudu.client.NonRecoverableException: Scanner 7595d00fd4cc4af0a085f256b5dc09a3 not found (it may have expired)",
"cause": {
"type": "org.apache.kudu.client.NonRecoverableException",
"message": "Scanner 7595d00fd4cc4af0a085f256b5dc09a3 not found (it may have expired)",
"suppressed": [
{
"type": "org.apache.kudu.client.KuduException.OriginalException",
"message": "Original asynchronous stack trace",
"suppressed": [],
"stack": [
"org.apache.kudu.client.RpcProxy.dispatchTSError(RpcProxy.java:355)",
"org.apache.kudu.client.RpcProxy.responseReceived(RpcProxy.java:277)",
"org.apache.kudu.client.RpcProxy.access$000(RpcProxy.java:63)",
"org.apache.kudu.client.RpcProxy$1.call(RpcProxy.java:157)",
"org.apache.kudu.client.RpcProxy$1.call(RpcProxy.java:153)",
"org.apache.kudu.client.Connection.channelRead0(Connection.java:368)",
"org.apache.kudu.shaded.io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:99)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)",
"org.apache.kudu.shaded.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324)",
"org.apache.kudu.shaded.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:296)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)",
"org.apache.kudu.shaded.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:324)",
"org.apache.kudu.shaded.io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:296)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357)",
"org.apache.kudu.shaded.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead(DefaultChannelPipeline.java:1410)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379)",
"org.apache.kudu.shaded.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365)",
"org.apache.kudu.shaded.io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:919)",
"org.apache.kudu.shaded.io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:166)",
"org.apache.kudu.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:719)",
"org.apache.kudu.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:655)",
"org.apache.kudu.shaded.io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:581)",
"org.apache.kudu.shaded.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:493)",
"org.apache.kudu.shaded.io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:989)",
"org.apache.kudu.shaded.io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74)",
"java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)",
"java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)",
"java.base/java.lang.Thread.run(Thread.java:829)"
],
"errorCode": {
"code": 65536,
"name": "GENERIC_INTERNAL_ERROR",
"type": "INTERNAL_ERROR"
}
}
],
"stack": [
"org.apache.kudu.client.KuduException.transformException(KuduException.java:110)",
"org.apache.kudu.client.KuduClient.joinAndHandleException(KuduClient.java:497)",
"org.apache.kudu.client.KuduScanner.keepAlive(KuduScanner.java:102)",
"org.apache.kudu.client.KuduScannerIterator.keepKuduScannerAlive(KuduScannerIterator.java:53)",
"org.apache.kudu.client.KuduScannerIterator.hasNext(KuduScannerIterator.java:74)",
"org.apache.kudu.client.KuduScannerIterator.hasNext(KuduScannerIterator.java:83)",
"io.trino.plugin.kudu.KuduRecordCursor.advanceNextPosition(KuduRecordCursor.java:89)",
"io.trino.$gen.CursorProcessor_20220916_000129_85581.process(Unknown Source)",
"io.trino.operator.ScanFilterAndProjectOperator$RecordCursorToPages.process(ScanFilterAndProjectOperator.java:338)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:240)",
"io.trino.operator.WorkProcessorUtils$YieldingProcess.process(WorkProcessorUtils.java:182)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:325)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.lambda$flatten$7(WorkProcessorUtils.java:296)",
"io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:338)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:325)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:240)",
"io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$3(WorkProcessorUtils.java:219)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:240)",
"io.trino.operator.WorkProcessorUtils.lambda$finishWhen$4(WorkProcessorUtils.java:234)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:150)",
"io.trino.operator.Driver.processInternal(Driver.java:410)",
"io.trino.operator.Driver.lambda$process$10(Driver.java:313)",
"io.trino.operator.Driver.tryWithLock(Driver.java:698)",
"io.trino.operator.Driver.process(Driver.java:305)",
"io.trino.operator.Driver.processForDuration(Driver.java:276)",
"io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1092)",
"io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)",
"io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:488)",
"io.trino.$gen.Trino_386_mdh_SNAPSHOT____20220915_112159_2.run(Unknown Source)",
"java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)",
"java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)",
"java.base/java.lang.Thread.run(Thread.java:829)"
],
"errorCode": {
"code": 65536,
"name": "GENERIC_INTERNAL_ERROR",
"type": "INTERNAL_ERROR"
}
},
"suppressed": [],
"stack": [
"org.apache.kudu.client.KuduScannerIterator.hasNext(KuduScannerIterator.java:77)",
"org.apache.kudu.client.KuduScannerIterator.hasNext(KuduScannerIterator.java:83)",
"io.trino.plugin.kudu.KuduRecordCursor.advanceNextPosition(KuduRecordCursor.java:89)",
"io.trino.$gen.CursorProcessor_20220916_000129_85581.process(Unknown Source)",
"io.trino.operator.ScanFilterAndProjectOperator$RecordCursorToPages.process(ScanFilterAndProjectOperator.java:338)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:240)",
"io.trino.operator.WorkProcessorUtils$YieldingProcess.process(WorkProcessorUtils.java:182)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:325)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.lambda$flatten$7(WorkProcessorUtils.java:296)",
"io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:338)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils$3.process(WorkProcessorUtils.java:325)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:240)",
"io.trino.operator.WorkProcessorUtils.lambda$processStateMonitor$3(WorkProcessorUtils.java:219)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorUtils.getNextState(WorkProcessorUtils.java:240)",
"io.trino.operator.WorkProcessorUtils.lambda$finishWhen$4(WorkProcessorUtils.java:234)",
"io.trino.operator.WorkProcessorUtils$ProcessWorkProcessor.process(WorkProcessorUtils.java:391)",
"io.trino.operator.WorkProcessorSourceOperatorAdapter.getOutput(WorkProcessorSourceOperatorAdapter.java:150)",
"io.trino.operator.Driver.processInternal(Driver.java:410)",
"io.trino.operator.Driver.lambda$process$10(Driver.java:313)",
"io.trino.operator.Driver.tryWithLock(Driver.java:698)",
"io.trino.operator.Driver.process(Driver.java:305)",
"io.trino.operator.Driver.processForDuration(Driver.java:276)",
"io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:1092)",
"io.trino.execution.executor.PrioritizedSplitRunner.process(PrioritizedSplitRunner.java:163)",
"io.trino.execution.executor.TaskExecutor$TaskRunner.run(TaskExecutor.java:488)",
"io.trino.$gen.Trino_386_mdh_SNAPSHOT____20220915_112159_2.run(Unknown Source)",
"java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)",
"java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)",
"java.base/java.lang.Thread.run(Thread.java:829)"
],
"errorCode": {
"code": 65536,
"name": "GENERIC_INTERNAL_ERROR",
"type": "INTERNAL_ERROR"
}
}
When KuduScanner's keepalive requests are not sent to tserver in time, the Scanner may be garbage-collected by tserver and will lead to this error. There is a similar issue reported in Kudu community, KUDU-3169. The error occurs when cluster load is high.
KuduScanner.keepalive() is only invoked in iteration when Trino is fetching data pages via KuduScannerIterator. When Kudu server is busy or Trino processes data slowly, scanner may have expired before next keepalive is sent.
The error can be re-produced in the following way:
- start a kudu server with short scanner TTL and gc-check interval, by configure tserver with options like
--scanner_ttl_ms=5000 --scanner_gc_check_interval_us=1000000 - simulate slow data fetching using the following code:
public void testScannerKeepalive()
throws Exception
{
List<Split> splits = getKuduSplits(); // get splits from QueryRunner's SplitManager
KuduSplit split = (KuduSplit) splits.get(0).getConnectorSplit();
KuduClientSession clientSession = createClientSession();
KuduRecordSet recordSet = new KuduRecordSet(clientSession, split, columnHandles);
try (RecordCursor cursor = recordSet.cursor()) {
cursor.advanceNextPosition();
// simulation of slow operation on result rows
TimeUnit.SECONDS.sleep(20);
// if scanner keepalive requests are not sent to tserver during the slow operation,
// the following code will raise an RuntimeException with a message like:
// "org.apache.kudu.client.NonRecoverableException: Scanner xxxxxxx not found (it may have expired)"
cursor.advanceNextPosition();
}
}
To fix this issue, we can:
- add Kudu scanner
keepAlivePeriodMsconfig for tuning and keepalive controlling - invoke
KuduScanner.keepaliveperiodically in a separated thread
@xiacongling Thanks for debugging this and finding the cause. Since you already have an understanding of what needs to be done would you like to send a PR?
I think having a background thread makes most sense since there's no good default value for the keepAlivePeriodMs (although we can increase the current value to make sure background thread doesn't need to work too frequently).
Hi, @hashhar . I will send a PR later. You can assign this issue to me.