This message was deleted.
# troubleshooting
s
This message was deleted.
n
Hi Oleg! Did you ever get erasure coding working with druid for deep storage? We’ve just started playing and we’re running into nullpointerexceptions when trying to write segments or log files to hdfs (3.3.6) on druid 27.0.0. If the hdfs directory is configured with standard replication, everything is fine. The moment we configure the directory to be erasure coded in hdfs, druid ceases to be able to write to it. Smells like a bug in druid (or the way it interacts with the files in hdfs (appending maybe))? - https://hadoop.apache.org/docs/r3.3.6/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations Anyone got erasure coding working with druid for deep storage?
k
I checked the code. Druid is not using append/truncate and concat for the hdfs segment pushing piece.
If you share the stack trace of the null pointer we can probably help more
n
Copy code
2023-09-11T09:17:06,188 INFO [forking-task-runner-0] org.apache.druid.indexing.overlord.ForkingTaskRunner - Exception caught during execution
java.lang.NullPointerException: null
at org.apache.hadoop.io.erasurecode.CodecUtil.createRawEncoderWithFallback(CodecUtil.java:174) ~[?:?]
at org.apache.hadoop.io.erasurecode.CodecUtil.createRawEncoder(CodecUtil.java:131) ~[?:?]
at org.apache.hadoop.hdfs.DFSStripedOutputStream.<init>(DFSStripedOutputStream.java:312) ~[?:?]
at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:315) ~[?:?]
at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1271) ~[?:?]
at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1250) ~[?:?]
at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1232) ~[?:?]
at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1170) ~[?:?]
at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:569) ~[?:?]
at org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:566) ~[?:?]
at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) ~[?:?]
at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:580) ~[?:?]
at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:507) ~[?:?]
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1233) ~[?:?]
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1210) ~[?:?]
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1091) ~[?:?]
at org.apache.druid.storage.hdfs.tasklog.HdfsTaskLogs.pushTaskFile(HdfsTaskLogs.java:92) ~[?:?]
at org.apache.druid.storage.hdfs.tasklog.HdfsTaskLogs.pushTaskLog(HdfsTaskLogs.java:65) ~[?:?]
at org.apache.druid.indexing.overlord.ForkingTaskRunner.waitForTaskProcessToComplete(ForkingTaskRunner.java:517) ~[druid-indexing-service-27.0.0.jar:27.0.0]
at org.apache.druid.indexing.overlord.ForkingTaskRunner$1.call(ForkingTaskRunner.java:404) ~[druid-indexing-service-27.0.0.jar:27.0.0]
at org.apache.druid.indexing.overlord.ForkingTaskRunner$1.call(ForkingTaskRunner.java:171) ~[druid-indexing-service-27.0.0.jar:27.0.0]
at java.util.concurrent.FutureTask.run(FutureTask.java:264) ~[?:?]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) ~[?:?]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) ~[?:?]
at java.lang.Thread.run(Thread.java:829) ~[?:?]
k
This looks more like 1. Either we are not using the correct version of HDFS client libs. Please check you are using the hadoop 3.3.6 binaries. 2. We need to set some codec information on the clients :
Copy code
The codec implementations for Reed-Solomon and XOR can be configured with the following client and DataNode configuration keys: io.erasurecode.codec.rs.rawcoders for the default RS codec, io.erasurecode.codec.rs-legacy.rawcoders for the legacy RS codec, io.erasurecode.codec.xor.rawcoders for the XOR codec. User can also configure self-defined codec with configuration key like: io.erasurecode.codec.self-defined-codec.rawcoders. The values for these key are lists of coder names with a fall-back mechanism. These codec factories are loaded in the order specified by the configuration values, until a codec is loaded successfully. The default RS and XOR codec configuration prefers native implementation over the pure Java one. There is no RS-LEGACY native codec implementation so the default is pure Java implementation only. All these codecs have implementations in pure Java. For default RS codec, there is also a native implementation which leverages Intel ISA-L library to improve the performance of codec. For XOR codec, a native implementation which leverages Intel ISA-L library to improve the performance of codec is also supported. Please refer to section "Enable Intel ISA-L" for more detail information. The default implementation for RS Legacy is pure Java, and the default implementations for default RS and XOR are native implementations using Intel ISA-L library.
I suggest create a simple hdfs client to push stuff to hdfs from the druid node. That will help you debug more.
p
Hi, I also have same problem. For me main reason was that pushing segment to hdfs first compressing from local node to
Copy code
<https://github.com/apache/druid/blob/master/extensions-core/hdfs-storage/src/main/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusher.java>:
    Path tmpIndexFile = new Path(StringUtils.format(
        "%s/%s/%s/%s_index.zip",
        fullyQualifiedStorageDirectory.get(),
        segment.getDataSource(),
        UUIDUtils.generateUuid(),
        segment.getShardSpec().getPartitionNum()
    ));
but because you cant append to files with EC enabled that was throwing an error. My solution was simple to change tmpIndexFile path to somewhere without EC. But probably there are more elegant solutions, maybe override EC policy for tmp files, compressing to local node,