Re: [I] [SUPPORT] Trino can't read tables created by Flink Hudi conector [hudi]

2023-10-26 Thread via GitHub


ramkrish86 commented on issue #9435:
URL: https://github.com/apache/hudi/issues/9435#issuecomment-1780744806

   @danny0405  - There are two stack traces in the above comment from 
@galadrielwithlaptop . What we can see is that with ABFS support or local 
system before the compaction could get completed there is an interrupted 
exception . But we can see that the Compaction job gets picked up in the Flink 
job graph but seems that before the compaction is over the parent thread is 
probably closed. When we manually run the compaction tool we are able to make 
it run. Is there something like a config that is missing over here? Any inputs 
could help us as from code perspective we are not able to see any issues over 
here. 


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [I] [SUPPORT] Trino can't read tables created by Flink Hudi conector [hudi]

2023-10-25 Thread via GitHub


danny0405 commented on issue #9435:
URL: https://github.com/apache/hudi/issues/9435#issuecomment-1780393004

   Were you capable to debug the local fs test failures?


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [I] [SUPPORT] Trino can't read tables created by Flink Hudi conector [hudi]

2023-10-25 Thread via GitHub


galadrielwithlaptop commented on issue #9435:
URL: https://github.com/apache/hudi/issues/9435#issuecomment-1778858927

   @danny0405 Local filesystem only - path was , file://home/walls/hive3t9


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [I] [SUPPORT] Trino can't read tables created by Flink Hudi conector [hudi]

2023-10-10 Thread via GitHub


danny0405 commented on issue #9435:
URL: https://github.com/apache/hudi/issues/9435#issuecomment-1756642742

   @galadrielwithlaptop Looks like there are many fs relating errors, in your 
local env, what filesystem did you use?


-- 
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.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [I] [SUPPORT] Trino can't read tables created by Flink Hudi conector [hudi]

2023-10-09 Thread via GitHub


galadrielwithlaptop commented on issue #9435:
URL: https://github.com/apache/hudi/issues/9435#issuecomment-1752691995

   Steps to repro:
   1.   We need to add calcite-core dependency in the server classpath. 
   2.   We need to follow this doc: 
https://hudi.apache.org/docs/syncing_metastore#flink-setup
   Which basically tells to use hive3, Hadoop3 profile compiled hudi jar and 
keep hudi-mr-bundle in Hive Metastore auxlib directory.
   
   PS: this is to get rid of Class Exceptions we got in the process.
   
   Commands:
   create catalog hive with(
 'type'='hudi',
 'mode'='hms',
 'hive.conf.dir'='/opt/hive-conf');

   USE CATALOG hive;
   
ADD JAR ‘hudi-flink-0.3.0-1.16.0.jar’;
   In order to create a new table.

   CREATE TABLE hive3t9(
  uuid VARCHAR(20),
  name VARCHAR(10),
  age INT,
  `partition` VARCHAR(20)
)
PARTITIONED BY (`partition`)
WITH (
  'connector' = 'hudi',
  'path' = 
'abfs://x...@xx.dfs.core.windows.net/hive3t9','hoodie.table.base.fi
   le.format' = 'PARQUET',
  'table.type' = 'MERGE_ON_READ',  -- If MERGE_ON_READ, hive query will not 
have output until th
   e parquet file is generated
  'hive_sync.enable' = 'true', -- Required. To enable hive 
synchronization
  'hive_sync.mode' = 'hms',-- Required. Setting hive sync mode to 
hms, default jdbc
  'hive_sync.metastore.uris' = 'thrift://hive-metastore:9083/' -- Required. 
The port need set on );
   
   Insert Data into Table:
   
   INSERT INTO hive3t8 VALUES ('8722', 'ehkjhdwumj', 87, 'Maths’);
   
   
   COW Tables: After these steps, cow just works fine and we are able to see 
data in Trino
   MOR Tables: We see the job as successful in Flink UI. The data also gets 
written to ABFS directory. But when Trino tries to read the data, it fails. And 
the catch is, the data gets compacted to a column-based parquet file format 
after every 5 commits. This compaction jobs fails. 
   
   As we dived into TM logs, we got these exceptions:
   
   
   **Caused by: java.lang.InterruptedException
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(Unknown
 Source) ~[?:?]
at java.util.concurrent.Semaphore.acquire(Unknown Source) ~[?:?]
at 
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3556) 
~[flink-hadoop-dep-1.16.0-0.0.18.jar:?]
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3521) 
~[flink-hadoop-dep-1.16.0-0.0.18.jar:?]
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:540) 
~[flink-hadoop-dep-1.16.0-0.0.18.jar:?]
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:365) 
~[flink-hadoop-dep-1.16.0-0.0.18.jar:?]**
   
   Solution to above problem:
   Solution:
   We upgraded Hadoop to 3.3.5 in JM/TM and compiled hudi-flink with the same 
version owing to https://issues.apache.org/jira/browse/HADOOP-17779
   
   Exception 2: 
   Exception 2:
   Coming out of the previous error, we got this. 
   
   
   **2023-09-21 06:11:22.564 [] pool-14-thread-1 ERROR flink 
apache.hudi.sink.compact.CompactOperator 140 Executor executes action [Execute 
compaction for instant 20230917184914418 from task 0] error
   org.apache.hudi.exception.HoodieInsertException: Failed to close the Insert 
Handle for path 
abfs://vrhudin...@vrstorageteam1.dfs.core.windows.net/hive3t8/Maths/cfe09673-7c1c-4e6c-9758-a44d531a1a03_0-1-0_20230917184914418.parquet
at 
org.apache.hudi.io.HoodieCreateHandle.close(HoodieCreateHandle.java:217) 
~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.table.HoodieFlinkCopyOnWriteTable.handleInsert(HoodieFlinkCopyOnWriteTable.java:408)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.table.action.compact.CompactionExecutionHelper.writeFileAndGetWriteStats(CompactionExecutionHelper.java:68)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:231)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.table.action.compact.HoodieCompactor.compact(HoodieCompactor.java:144)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.sink.compact.CompactOperator.doCompaction(CompactOperator.java:136)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.sink.compact.CompactOperator.lambda$processElement$0(CompactOperator.java:119)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at 
org.apache.hudi.sink.utils.NonThrownExecutor.lambda$wrapAction$0(NonThrownExecutor.java:130)
 ~[hudi-flink1.16-bundle-0.13.0.jar:0.13.0]
at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) 
[?:?]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) 
[?:?]
at java.lang.Thread.run(Unknown Source) [?:?]
   Caused by: java.io.IOException: Failed with java.io.IOException while 
processing file/directory