hudi-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [incubator-hudi] lamber-ken commented on issue #1373: [WIP] [HUDI-646] Re-enable TestUpdateSchemaEvolution to reproduce CI error
Date Mon, 09 Mar 2020 17:06:26 GMT
lamber-ken commented on issue #1373: [WIP] [HUDI-646] Re-enable TestUpdateSchemaEvolution to
reproduce CI error
URL: https://github.com/apache/incubator-hudi/pull/1373#issuecomment-596656326
 
 
   hi @vinothchandar 
   
   After countless tests, I finally found the answer. Each test needs to wait a lot of time
for the CI to build. Let me explain this challenging issue step by step.
   
   Three questions
   1, Why ci build failure after code cleanup?
   2, Why junit doesn't report failure in local env?
   3, Why `TestHBaseQPSResourceAllocator` affects `TestUpdateSchemaEvolution`?
   
   
   #### Stackstrace
   ```
   Job aborted due to stage failure: Task 7 in stage 1.0 failed 1 times, most recent failure:
Lost task 7.0 in stage 1.0 (TID 15, localhost, executor driver): org.apache.parquet.io.ParquetDecodingException:
Can not read value at 0 in block -1 in file file:/tmp/junit3406952253616234024/2016/01/31/f1-0_7-0-7_100.parquet
   	at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:251)
   	at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:132)
   	at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:136)
   	at org.apache.hudi.common.util.ParquetUtils.readAvroRecords(ParquetUtils.java:190)
   	at org.apache.hudi.client.TestUpdateSchemaEvolution.lambda$testSchemaEvolutionOnUpdate$dfb2f24e$1(TestUpdateSchemaEvolution.java:123)
   	at org.apache.spark.api.java.JavaPairRDD$$anonfun$toScalaFunction$1.apply(JavaPairRDD.scala:1040)
   	at scala.collection.Iterator$$anon$11.next(Iterator.scala:410)
   	at scala.collection.Iterator$class.foreach(Iterator.scala:891)
   	at scala.collection.AbstractIterator.foreach(Iterator.scala:1334)
   	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
   	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
   	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
   	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:310)
   	at scala.collection.AbstractIterator.to(Iterator.scala:1334)
   	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:302)
   	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1334)
   	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:289)
   	at scala.collection.AbstractIterator.toArray(Iterator.scala:1334)
   	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:945)
   	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$13.apply(RDD.scala:945)
   	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
   	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:2101)
   	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
   	at org.apache.spark.scheduler.Task.run(Task.scala:123)
   	at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
   	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
   	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
   	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   	at java.lang.Thread.run(Thread.java:748)
   Caused by: java.lang.UnsupportedOperationException: Byte-buffer read unsupported by input
stream
   	at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:146)
   	at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:143)
   	at org.apache.parquet.hadoop.util.H2SeekableInputStream$H2Reader.read(H2SeekableInputStream.java:81)
   	at org.apache.parquet.hadoop.util.H2SeekableInputStream.readFully(H2SeekableInputStream.java:90)
   	at org.apache.parquet.hadoop.util.H2SeekableInputStream.readFully(H2SeekableInputStream.java:75)
   	at org.apache.parquet.hadoop.ParquetFileReader$ConsecutiveChunkList.readAll(ParquetFileReader.java:1174)
   	at org.apache.parquet.hadoop.ParquetFileReader.readNextRowGroup(ParquetFileReader.java:805)
   	at org.apache.parquet.hadoop.InternalParquetRecordReader.checkRead(InternalParquetRecordReader.java:127)
   	at org.apache.parquet.hadoop.InternalParquetRecordReader.nextKeyValue(InternalParquetRecordReader.java:222)
   	... 29 more
   ```
   
   **Step 1:** Check whether the `f1-0_7-0-7_100.parquet` file is complete or not, used hdfs
api to check it, it's no problem
   
   **Setp 2:** Noticed the `UnsupportedOperationException: Byte-buffer read unsupported by
input stream` exception
   Add some log statements to `FSDataInputStream` and rerun the unit by travis, I found two
different implementation class
   
   - org.apache.hadoop.fs.FSDataInputStream
   - org.apache.hadoop.fs.ChecksumFileSystem.ChecksumFSInputChecker
   
   ```
   class org.apache.hadoop.fs.FSDataInputStream
   java.lang.Exception
   	at org.apache.hadoop.fs.FSDataInputStream.<init>(FSDataInputStream.java:55)
   	at org.apache.hadoop.fs.ChecksumFileSystem$FSDataBoundedInputStream.<init>(ChecksumFileSystem.java:271)
   	at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:351)
   	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:769)
   	at org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:65)
   	at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:688)
   	at org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:596)
   	at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:152)
   	at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
   ```
   ```
   class org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSInputChecker
   java.lang.Exception
   	at org.apache.hadoop.fs.FSDataInputStream.<init>(FSDataInputStream.java:55)
   	at org.apache.hadoop.fs.ChecksumFileSystem$FSDataBoundedInputStream.<init>(ChecksumFileSystem.java:271)
   	at org.apache.hadoop.fs.ChecksumFileSystem.open(ChecksumFileSystem.java:351)
   	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:769)
   	at org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:65)
   	at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:688)
   	at org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:596)
   	at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:152)
   	at org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
   ```
   
   **Step 3:** why exits two different stackstrace when read parquet file?
   from `org.apache.hadoop.fs.ChecksumFileSystem#open`, we can see that it controlled by `verifyChecksum`
variable
   ```
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
     FileSystem fs;
     InputStream in;
     if (verifyChecksum) {
       fs = this;
       in = new ChecksumFSInputChecker(this, f, bufferSize);
     } else {
       fs = getRawFileSystem();
       in = fs.open(f, bufferSize);
     }
     return new FSDataBoundedInputStream(fs, f, in);
   }
   ```
   
   **Step 4:** debug when `verifyChecksum` changed? `verifyChecksum` by default is `true`.
   
   while `new HBaseTestingUtility()` in `org.apache.hudi.index.TestHBaseQPSResourceAllocator`
   ```
   java.lang.Exception
   	at org.apache.hadoop.fs.ChecksumFileSystem.setWriteChecksum(ChecksumFileSystem.java:79)
   	at org.apache.hadoop.hbase.fs.HFileSystem.<init>(HFileSystem.java:87)
   	at org.apache.hadoop.hbase.fs.HFileSystem.get(HFileSystem.java:372)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.getTestFileSystem(HBaseTestingUtility.java:3058)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.getNewDataTestDirOnTestFS(HBaseTestingUtility.java:496)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.setupDataTestDirOnTestFS(HBaseTestingUtility.java:485)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.getDataTestDirOnTestFS(HBaseTestingUtility.java:458)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.getDataTestDirOnTestFS(HBaseTestingUtility.java:472)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.createDirsAndSetProperties(HBaseTestingUtility.java:649)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniDFSCluster(HBaseTestingUtility.java:575)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniCluster(HBaseTestingUtility.java:982)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniCluster(HBaseTestingUtility.java:863)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniCluster(HBaseTestingUtility.java:857)
   	at org.apache.hadoop.hbase.HBaseTestingUtility.startMiniCluster(HBaseTestingUtility.java:801)
   ```
   ```
   public HFileSystem(Configuration conf, boolean useHBaseChecksum)
     throws IOException {
   
     ...
     
     // disable checksum verification for local fileSystem, see HBASE-11218
     if (fs instanceof LocalFileSystem) {
       fs.setWriteChecksum(false);
       fs.setVerifyChecksum(false);
     }
   
     ...
   }  
   ```
   
   **Step 5:** why `TestHBaseQPSResourceAllocator` affect `TestUpdateSchemaEvolution`?
   
   From the following code, we know that `filesystem` is shared with same uri, 
   or please check `org.apache.hadoop.fs.FileSystem`
   
   ```
   verifyChecksum: false
   fs1 hashcode: 1288235781
   fs2 hashcode: 1288235781
   
   FileSystem fs1 = new Path("aaa").getFileSystem(new Configuration());
   fs1.setVerifyChecksum(false);
   
   FileSystem fs2 = new Path("bbb").getFileSystem(new Configuration());
   if (fs2 instanceof ChecksumFileSystem) {
     Field field = ChecksumFileSystem.class.getDeclaredField("verifyChecksum");
     field.setAccessible(true);
     System.out.println("verifyChecksum: " + field.get(fs2));
   }
   
   System.out.println("fs1 hashcode: " + fs1.hashCode());
   System.out.println("fs2 hashcode: " + fs1.hashCode());
   ```
   
   **Step 6:** why ci build failure after code cleanup?
   
   we know junit executes in alphabetical order, so after cleanup, ci failure
   
   prev 
   - TestHBaseQPSResourceAllocator
   - TestUpdateMapFunction
   
   now
   - TestHBaseQPSResourceAllocator
   - TestUpdateSchemaEvolution
   
   **Step 7:** Why junit doesn't report failure in local env?
   
   Because we just test `TestUpdateSchemaEvolution`, the `verifyChecksum` doesn't be modified.
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message