[ 
https://issues.apache.org/jira/browse/HIVE-21146?focusedWorklogId=345634&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-345634
 ]

ASF GitHub Bot logged work on HIVE-21146:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 18/Nov/19 22:26
            Start Date: 18/Nov/19 22:26
    Worklog Time Spent: 10m 
      Work Description: prasanthj commented on pull request #797: HIVE-21146 
Enforce TransactionBatch size=1 for blob stores
URL: https://github.com/apache/hive/pull/797#discussion_r347642281
 
 

 ##########
 File path: 
streaming/src/java/org/apache/hive/streaming/HiveStreamingConnection.java
 ##########
 @@ -520,6 +523,28 @@ private void validateTable() throws InvalidTable, 
ConnectionError {
       LOG.error(errMsg);
       throw new ConnectionError(errMsg);
     }
+
+    // batch size is only used for managed transactions, not for unmanaged 
single transactions
+    if (transactionBatchSize > 1) {
+      try (FileSystem fs = tableObject.getDataLocation().getFileSystem(conf)) {
+        if (BlobStorageUtils.isBlobStorageFileSystem(conf, fs)) {
+          // currently not all filesystems implement StreamCapabilities, while 
FSDataOutputStream does
+          Path path = new Path(tableObject.getDataLocation(), 
"tmp_stream_verify_" + UUID.randomUUID().toString());
 
 Review comment:
   I am not sure if putting files under table location is a good idea. Better 
place would be use the /tmp location but if that is not accessible for some 
reason atleast the files created under the table location should start with "." 
or "_" so that file listing ignores such files. 
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 345634)
    Time Spent: 0.5h  (was: 20m)

> Enforce TransactionBatch size=1 for blob stores
> -----------------------------------------------
>
>                 Key: HIVE-21146
>                 URL: https://issues.apache.org/jira/browse/HIVE-21146
>             Project: Hive
>          Issue Type: Bug
>          Components: Streaming, Transactions
>    Affects Versions: 3.0.0
>            Reporter: Eugene Koifman
>            Assignee: David Lavati
>            Priority: Major
>              Labels: pull-request-available
>         Attachments: HIVE-21146.2.patch, HIVE-21146.2.patch, 
> HIVE-21146.2.patch, HIVE-21146.2.patch, HIVE-21146.patch
>
>          Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> Streaming Ingest API supports a concept of {{TransactionBatch}} where N 
> transactions can be opened at once and the data in all of them will be 
> written to the same delta_x_y directory where each transaction in the batch 
> can be committed/aborted independently.  The implementation relies on 
> {{FSDataOutputStream.hflush()}} (called from OrcRecordUpdater}} which is 
> available on HDFS but is often implemented as no-op in Blob store backed 
> {{FileSystem}} objects.
> Need to add a check to {{HiveStreamingConnection()}} constructor to raise an 
> error if {{builder.transactionBatchSize > 1}} and the target table/partitions 
> are backed by something that doesn't support {{hflush()}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to