由于我们从 ipv4 换成 ipv6,需要测试一下 trino 的 hive catalog 是否支持 ipv6,本文记录了测试过程中遇到的问题以及解决办法。
从创建 hive catalog 说起
CREATE CATALOG hive_v6 USING hive
WITH (
"hive.config.resources" = '/home/trino/core-site.xml,/home/trino/hdfs-site.xml',
"hive.metastore.uri" = 'thrift://[2400:300:1200:0:0:0:0:73]:9083'
);
select * from hive_v6.sjzt.demo limit 10
'[2400' is not an IP string literal
分析报错日志
1. 详细报错日志
trino 的 server.log 拉取错误日志,如下:2024-07-06T13:37:42.817+0800 DEBUG SplitRunner-1 io.trino.execution.executor.dedicated.SplitProcessor Error opening Hive split hdfs://hdpv6/user/hive/demo/p_hour=202310090600/demo.csv (offset=0, length=2729): '[2400' is not an IP string literal.
io.trino.spi.TrinoException: Error opening Hive split hdfs://hdpv6/user/hive/demo/p_hour=202310090600/demo.csv (offset=0, length=2729): '[2400' is not an IP string literal.
at io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:156)
at io.trino.plugin.hive.HivePageSourceProvider.createHivePageSource(HivePageSourceProvider.java:200)
at io.trino.plugin.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:136)
at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:48)
at io.trino.split.PageSourceManager$PageSourceProviderInstance.createPageSource(PageSourceManager.java:79)
at io.trino.operator.TableScanOperator.getOutput(TableScanOperator.java:265)
at io.trino.operator.Driver.processInternal(Driver.java:403)
at io.trino.operator.Driver.lambda$process$8(Driver.java:306)
at io.trino.operator.Driver.tryWithLock(Driver.java:709)
at io.trino.operator.Driver.process(Driver.java:298)
at io.trino.operator.Driver.processForDuration(Driver.java:269)
at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890)
at io.trino.execution.executor.dedicated.SplitProcessor.run(SplitProcessor.java:77)
at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.lambda$run$0(TaskEntry.java:191)
at io.trino.$gen.Trino_449____20240703_100640_2.run(Unknown Source)
at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.run(TaskEntry.java:192)
at io.trino.execution.executor.scheduler.FairScheduler.runTask(FairScheduler.java:168)
at io.trino.execution.executor.scheduler.FairScheduler.lambda$submit$0(FairScheduler.java:155)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76)
at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
at java.base/java.lang.Thread.run(Thread.java:1570)
Caused by: java.lang.IllegalArgumentException: '[2400' is not an IP string literal.
at io.trino.hadoop.$internal.org.apache.hadoop.thirdparty.com.google.common.net.InetAddresses.formatIllegalArgumentException(InetAddresses.java:1123)
at io.trino.hadoop.$internal.org.apache.hadoop.thirdparty.com.google.common.net.InetAddresses.forString(InetAddresses.java:146)
at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.getPeerAddress(DataTransferSaslUtil.java:164)
at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:170)
at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:731)
at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3035)
at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:829)
at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:754)
at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:381)
at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:715)
at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:645)
at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:845)
at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:918)
at java.base/java.io.DataInputStream.read(DataInputStream.java:158)
at java.base/java.io.DataInputStream.read(DataInputStream.java:158)
at io.trino.filesystem.hdfs.HdfsTrinoInputStream.read(HdfsTrinoInputStream.java:102)
at java.base/java.io.InputStream.readNBytes(InputStream.java:412)
at java.base/java.io.InputStream.readAllBytes(InputStream.java:349)
at io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:140)
... 24 more
2. trino-449 源码分析
io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:156)










3. trino-hadoop-apache 源码分析
hadoop-apache 是 trino 和 hadoop 建立关系的一个依赖。hadoop-apache-3.3.5-3 的源码,代码下载链接为:https://github.com/trinodb/trino-hadoop-apache/archive/refs/tags/3.3.5-3.zip'[2400' is not an IP string literal是在下面方法中抛出来的:com.google.common.net.InetAddresses.formatIllegalArgumentException(InetAddresses.java:1123)
InetAddresses.java这个类:
guava-33.1.0-jre.jar的InetAddresses.class的forString方法中。org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.getPeerAddress(DataTransferSaslUtil.java:164)
forString方法是从hadoop-hdfs-client-3.3.5.jar 的DataTransferSaslUtil.java类的getPeerAddress方法调用的:
4. hadoop-client 源码分析
hadoop 的源码,接着分析,hadoop 源码下载链接为:https://github.com/apache/hadoop/archive/refs/tags/rel/release-3.3.5.zipDataTransferSaslUtil.java :

问题初步解决
hadoop-client打包。hadoop-rel-release-3.3.5/hadoop-hdfs-project/hadoop-hdfs-client目录,执行打包命令:mvn clean install -DskipTests
hadoop-hdfs-client-3.3.5.jar,然后将本地 maven 仓库的 jar 包替换成新编译的这个 jar 包:
trino-hadoop-apache-3.3.5-3项目:mvn clean package -DskipTests
hadoop-apache-3.3.5-3.jar 上传到 trino 部署目录的 trino-server-449/plugin/hive/hdfs,即:将 hive 插件的 hadoop-apache-3.3.5-3.jar替换成我们新编译的 jar 包。./bin/launcher restart
select * from hive_v6.sjzt."demo" limit 10;
出现新的问题
1. 详细报错日志
2024-07-06T16:57:46.258+0800 DEBUG SplitRunner-5 io.trino.execution.executor.dedicated.SplitProcessor Error opening Hive split hdfs://hdpv6/user/hive/demo/p_hour=202310090600/demo.csv (offset=0, length=2729): Does not contain a valid host:port authority: 2400:300:1200:0:0:0:0:73:50010
io.trino.spi.TrinoException: Error opening Hive split hdfs://hdpv6/user/hive/demo/p_hour=202310090600/demo.csv (offset=0, length=2729): Does not contain a valid host:port authority: 2400:300:1200:0:0:0:0:73:50010
at io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:157)
at io.trino.plugin.hive.HivePageSourceProvider.createHivePageSource(HivePageSourceProvider.java:200)
at io.trino.plugin.hive.HivePageSourceProvider.createPageSource(HivePageSourceProvider.java:136)
at io.trino.plugin.base.classloader.ClassLoaderSafeConnectorPageSourceProvider.createPageSource(ClassLoaderSafeConnectorPageSourceProvider.java:48)
at io.trino.split.PageSourceManager$PageSourceProviderInstance.createPageSource(PageSourceManager.java:79)
at io.trino.operator.TableScanOperator.getOutput(TableScanOperator.java:265)
at io.trino.operator.Driver.processInternal(Driver.java:403)
at io.trino.operator.Driver.lambda$process$8(Driver.java:306)
at io.trino.operator.Driver.tryWithLock(Driver.java:709)
at io.trino.operator.Driver.process(Driver.java:298)
at io.trino.operator.Driver.processForDuration(Driver.java:269)
at io.trino.execution.SqlTaskExecution$DriverSplitRunner.processFor(SqlTaskExecution.java:890)
at io.trino.execution.executor.dedicated.SplitProcessor.run(SplitProcessor.java:77)
at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.lambda$run$0(TaskEntry.java:191)
at io.trino.$gen.Trino_449____20240706_085653_2.run(Unknown Source)
at io.trino.execution.executor.dedicated.TaskEntry$VersionEmbedderBridge.run(TaskEntry.java:192)
at io.trino.execution.executor.scheduler.FairScheduler.runTask(FairScheduler.java:168)
at io.trino.execution.executor.scheduler.FairScheduler.lambda$submit$0(FairScheduler.java:155)
at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:572)
at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:131)
at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:76)
at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:82)
at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
at java.base/java.lang.Thread.run(Thread.java:1570)
Caused by: java.lang.IllegalArgumentException: Does not contain a valid host:port authority: 2400:300:1200:0:0:0:0:73:50010
at org.apache.hadoop.net.NetUtils.createSocketAddr(NetUtils.java:229)
at org.apache.hadoop.net.NetUtils.createSocketAddr(NetUtils.java:185)
at org.apache.hadoop.net.NetUtils.createSocketAddr(NetUtils.java:164)
at org.apache.hadoop.net.NetUtils.createSocketAddr(NetUtils.java:149)
at org.apache.hadoop.hdfs.ClientContext.getNetworkDistance(ClientContext.java:300)
at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReader(BlockReaderFactory.java:862)
at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:758)
at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:382)
at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:715)
at org.apache.hadoop.hdfs.DFSInputStream.blockSeekTo(DFSInputStream.java:645)
at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:845)
at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:918)
at java.base/java.io.DataInputStream.read(DataInputStream.java:158)
at java.base/java.io.DataInputStream.read(DataInputStream.java:158)
at io.trino.filesystem.hdfs.HdfsTrinoInputStream.read(HdfsTrinoInputStream.java:106)
at java.base/java.io.InputStream.readNBytes(InputStream.java:412)
at java.base/java.io.InputStream.readAllBytes(InputStream.java:349)
at io.trino.plugin.hive.line.LinePageSourceFactory.createPageSource(LinePageSourceFactory.java:135)
... 24 more
2. Hadoop-common 源码分析
NetUtils.java,这是hadoop-common的一个类:

createURI方法,可以看到,是这行代码解析 ipv6 出错的:

hadoop-common 替换成重新编译后的 hadoop-common-3.3.5.jar
trino-hadoop-apache-3.3.5-3项目:mvn clean package -DskipTests
3. 替换 hadoop-apache,重启 trino
hadoop-apache-3.3.5-3.jar 上传到 trino 部署目录的 trino-server-449/plugin/hive/hdfs,即:将 hive 插件的 hadoop-apache-3.3.5-3.jar替换成我们新编译的 jar 包。./bin/launcher restart
问题终于解决
select * from hive_v6.sjzt."demo" limit 10;
写在最后
当然,用到 hadoop 的插件都可以这样解决,比如 iceberg,hudi。
最终解决办法就是:
① 修改 hadoop-hdfs-client 项目的 DataTransferSaslUtil.java类:
public static InetAddress getPeerAddress(Peer peer) {
// String remoteAddr = peer.getRemoteAddressString().split(":")[0];
// 解决ipv6问题
String remoteAddressString = peer.getRemoteAddressString();
String remoteAddr = remoteAddressString.substring(0, remoteAddressString.lastIndexOf(":"))
.replace("[","").replace("]","");
int slashIdx = remoteAddr.indexOf('/');
return InetAddresses.forString(slashIdx != -1 ?
remoteAddr.substring(slashIdx + 1, remoteAddr.length()) :
remoteAddr);
}
hadoop-common 项目的 NetUtils.java类:private static URI createURI(String target,
boolean hasScheme,
String helpText,
boolean useCacheIfPresent) {
URI uri;
if (useCacheIfPresent) {
uri = URI_CACHE.getIfPresent(target);
if (uri != null) {
return uri;
}
}
try {
// 处理ipv6:2400:300:1200:0:0:0:0:73:50010
String[] targetArr = target.split(":");
if (targetArr.length > 8) {
String v6Host = target.substring(0, target.lastIndexOf(":"));
String v6Port = target.substring(target.lastIndexOf(":")+1);
target = String.format("[%s]:%s", v6Host, v6Port);
}
uri = hasScheme ? URI.create(target) :
URI.create("dummyscheme://" + target);
} catch (IllegalArgumentException e) {
throw new IllegalArgumentException(
"Does not contain a valid host:port authority: " + target + helpText
);
}
if (useCacheIfPresent) {
URI_CACHE.put(target, uri);
}
return uri;
}
./mvnw -pl '!:trino-server-rpm,!:trino-docs,!:trino-proxy,!:trino-verifier,!:trino-benchto-benchmarks' clean install -T10C -nsu -DskipTests -Dmaven.javadoc.skip=true -Dmaven.source.skip=true -Dair.check.skip-all=true -Dskip.npm -Dskip.yarn -Dmaven.gitcommitid.skip=true
往期推荐