[jira] [Commented] (PARQUET-2297) Encrypted files should not be checked for delta encoding problem

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719638#comment-17719638
 ] 

ASF GitHub Bot commented on PARQUET-2297:
-

ggershinsky commented on PR #1089:
URL: https://github.com/apache/parquet-mr/pull/1089#issuecomment-1535733579

   SGTM, I'll send a PR to the parquet-1.13.x branch too




> Encrypted files should not be checked for delta encoding problem
> 
>
> Key: PARQUET-2297
> URL: https://issues.apache.org/jira/browse/PARQUET-2297
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Gidon Gershinsky
>Assignee: Gidon Gershinsky
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> Delta encoding problem (https://issues.apache.org/jira/browse/PARQUET-246) 
> was fixed in writers since parquet-mr-1.8. This fix also added a 
> `checkDeltaByteArrayProblem` method in readers, that runs over all columns 
> and checks for this problem in older files. 
> This now triggers an unrelated exception when reading encrypted files, in the 
> following situation: trying to read an unencrypted column, without having 
> keys for encrypted columns (see 
> https://issues.apache.org/jira/browse/PARQUET-2193). This happens in Spark, 
> with nested columns (files with regular columns are ok).
> Possible solution: don't call the `checkDeltaByteArrayProblem` method for 
> encrypted files - because these files can be written only with 
> parquet-mr-1.12 and newer, where the delta encoding problem is already fixed.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] ggershinsky commented on pull request #1089: PARQUET-2297: Skip delta problem check for encrypted files

2023-05-04 Thread via GitHub


ggershinsky commented on PR #1089:
URL: https://github.com/apache/parquet-mr/pull/1089#issuecomment-1535733579

   SGTM, I'll send a PR to the parquet-1.13.x branch too


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2297) Encrypted files should not be checked for delta encoding problem

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719593#comment-17719593
 ] 

ASF GitHub Bot commented on PARQUET-2297:
-

wgtmac commented on PR #1089:
URL: https://github.com/apache/parquet-mr/pull/1089#issuecomment-1535610371

   Should we include this fix to the next 1.13.1 release: 
https://lists.apache.org/thread/1mjvdcmwqjcblmfkfgpd9ob2yodx7tom ?  
@ggershinsky 




> Encrypted files should not be checked for delta encoding problem
> 
>
> Key: PARQUET-2297
> URL: https://issues.apache.org/jira/browse/PARQUET-2297
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Gidon Gershinsky
>Assignee: Gidon Gershinsky
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> Delta encoding problem (https://issues.apache.org/jira/browse/PARQUET-246) 
> was fixed in writers since parquet-mr-1.8. This fix also added a 
> `checkDeltaByteArrayProblem` method in readers, that runs over all columns 
> and checks for this problem in older files. 
> This now triggers an unrelated exception when reading encrypted files, in the 
> following situation: trying to read an unencrypted column, without having 
> keys for encrypted columns (see 
> https://issues.apache.org/jira/browse/PARQUET-2193). This happens in Spark, 
> with nested columns (files with regular columns are ok).
> Possible solution: don't call the `checkDeltaByteArrayProblem` method for 
> encrypted files - because these files can be written only with 
> parquet-mr-1.12 and newer, where the delta encoding problem is already fixed.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] wgtmac commented on pull request #1089: PARQUET-2297: Skip delta problem check for encrypted files

2023-05-04 Thread via GitHub


wgtmac commented on PR #1089:
URL: https://github.com/apache/parquet-mr/pull/1089#issuecomment-1535610371

   Should we include this fix to the next 1.13.1 release: 
https://lists.apache.org/thread/1mjvdcmwqjcblmfkfgpd9ob2yodx7tom ?  
@ggershinsky 


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2254) Build a BloomFilter with a more precise size

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2254?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719591#comment-17719591
 ] 

ASF GitHub Bot commented on PARQUET-2254:
-

wgtmac commented on code in PR #1042:
URL: https://github.com/apache/parquet-mr/pull/1042#discussion_r1185642897


##
parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/AdaptiveBlockSplitBloomFilter.java:
##
@@ -38,57 +38,51 @@
 import org.apache.parquet.io.api.Binary;
 
 /**
- * `DynamicBlockBloomFilter` contains multiple `BlockSplitBloomFilter` as 
candidates and inserts values in
+ * `AdaptiveBlockSplitBloomFilter` contains multiple `BlockSplitBloomFilter` 
as candidates and inserts values in
  * the candidates at the same time.
  * The purpose of this is to finally generate a bloom filter with the optimal 
bit size according to the number
  * of real data distinct values. Use the largest bloom filter as an 
approximate deduplication counter, and then
  * remove incapable bloom filter candidate during data insertion.
  */
-public class DynamicBlockBloomFilter implements BloomFilter {
+public class AdaptiveBlockSplitBloomFilter implements BloomFilter {
 
-  private static final Logger LOG = 
LoggerFactory.getLogger(DynamicBlockBloomFilter.class);
+  private static final Logger LOG = 
LoggerFactory.getLogger(AdaptiveBlockSplitBloomFilter.class);
 
   // multiple candidates, inserting data at the same time. If the distinct 
values are greater than the
   // expected NDV of candidates, it will be removed. Finally we will choose 
the smallest candidate to write out.
   private final List candidates = new ArrayList<>();
 
   // the largest among candidates and used as an approximate deduplication 
counter
-  private BloomFilterCandidate maxCandidate;
+  private BloomFilterCandidate largestCandidate;
 
   // the accumulator of the number of distinct values that have been inserted 
so far
-  private int distinctValueCounter = 0;
+  private long distinctValueCounter = 0;
 
   // indicates that the bloom filter candidate has been written out and new 
data should be no longer allowed to be inserted
   private boolean finalized = false;
 
+  // indicates the step size to find the NDV value corresponding to numBytes
+  private static final int NDV_STEP = 500;
   private int maximumBytes = UPPER_BOUND_BYTES;
   private int minimumBytes = LOWER_BOUND_BYTES;
   // the hash strategy used in this bloom filter.
   private final HashStrategy hashStrategy;
   // the column to build bloom filter
   private ColumnDescriptor column;
 
-  public DynamicBlockBloomFilter(int numBytes, int candidatesNum, double fpp, 
ColumnDescriptor column) {
-this(numBytes, LOWER_BOUND_BYTES, UPPER_BOUND_BYTES, HashStrategy.XXH64, 
fpp, candidatesNum, column);
-  }
-
-  public DynamicBlockBloomFilter(int numBytes, int maximumBytes, int 
candidatesNum, double fpp, ColumnDescriptor column) {
-this(numBytes, LOWER_BOUND_BYTES, maximumBytes, HashStrategy.XXH64, fpp, 
candidatesNum, column);
+  /**
+   * Given the maximum acceptable bytes size of bloom filter, generate 
candidates according it.

Review Comment:
   ```suggestion
  * Generate bloom filter candidates according to the maximum acceptable 
byte size.
   ```



##
parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/AdaptiveBlockSplitBloomFilter.java:
##
@@ -38,57 +38,51 @@
 import org.apache.parquet.io.api.Binary;
 
 /**
- * `DynamicBlockBloomFilter` contains multiple `BlockSplitBloomFilter` as 
candidates and inserts values in
+ * `AdaptiveBlockSplitBloomFilter` contains multiple `BlockSplitBloomFilter` 
as candidates and inserts values in
  * the candidates at the same time.
  * The purpose of this is to finally generate a bloom filter with the optimal 
bit size according to the number
  * of real data distinct values. Use the largest bloom filter as an 
approximate deduplication counter, and then
  * remove incapable bloom filter candidate during data insertion.
  */
-public class DynamicBlockBloomFilter implements BloomFilter {
+public class AdaptiveBlockSplitBloomFilter implements BloomFilter {
 
-  private static final Logger LOG = 
LoggerFactory.getLogger(DynamicBlockBloomFilter.class);
+  private static final Logger LOG = 
LoggerFactory.getLogger(AdaptiveBlockSplitBloomFilter.class);
 
   // multiple candidates, inserting data at the same time. If the distinct 
values are greater than the
   // expected NDV of candidates, it will be removed. Finally we will choose 
the smallest candidate to write out.
   private final List candidates = new ArrayList<>();
 
   // the largest among candidates and used as an approximate deduplication 
counter
-  private BloomFilterCandidate maxCandidate;
+  private BloomFilterCandidate largestCandidate;
 
   // the accumulator of the number of distinct values that have been inserted 
so far
-  private int 

[GitHub] [parquet-mr] wgtmac commented on a diff in pull request #1042: PARQUET-2254 Support building dynamic bloom filter that adapts to the data

2023-05-04 Thread via GitHub


wgtmac commented on code in PR #1042:
URL: https://github.com/apache/parquet-mr/pull/1042#discussion_r1185642897


##
parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/AdaptiveBlockSplitBloomFilter.java:
##
@@ -38,57 +38,51 @@
 import org.apache.parquet.io.api.Binary;
 
 /**
- * `DynamicBlockBloomFilter` contains multiple `BlockSplitBloomFilter` as 
candidates and inserts values in
+ * `AdaptiveBlockSplitBloomFilter` contains multiple `BlockSplitBloomFilter` 
as candidates and inserts values in
  * the candidates at the same time.
  * The purpose of this is to finally generate a bloom filter with the optimal 
bit size according to the number
  * of real data distinct values. Use the largest bloom filter as an 
approximate deduplication counter, and then
  * remove incapable bloom filter candidate during data insertion.
  */
-public class DynamicBlockBloomFilter implements BloomFilter {
+public class AdaptiveBlockSplitBloomFilter implements BloomFilter {
 
-  private static final Logger LOG = 
LoggerFactory.getLogger(DynamicBlockBloomFilter.class);
+  private static final Logger LOG = 
LoggerFactory.getLogger(AdaptiveBlockSplitBloomFilter.class);
 
   // multiple candidates, inserting data at the same time. If the distinct 
values are greater than the
   // expected NDV of candidates, it will be removed. Finally we will choose 
the smallest candidate to write out.
   private final List candidates = new ArrayList<>();
 
   // the largest among candidates and used as an approximate deduplication 
counter
-  private BloomFilterCandidate maxCandidate;
+  private BloomFilterCandidate largestCandidate;
 
   // the accumulator of the number of distinct values that have been inserted 
so far
-  private int distinctValueCounter = 0;
+  private long distinctValueCounter = 0;
 
   // indicates that the bloom filter candidate has been written out and new 
data should be no longer allowed to be inserted
   private boolean finalized = false;
 
+  // indicates the step size to find the NDV value corresponding to numBytes
+  private static final int NDV_STEP = 500;
   private int maximumBytes = UPPER_BOUND_BYTES;
   private int minimumBytes = LOWER_BOUND_BYTES;
   // the hash strategy used in this bloom filter.
   private final HashStrategy hashStrategy;
   // the column to build bloom filter
   private ColumnDescriptor column;
 
-  public DynamicBlockBloomFilter(int numBytes, int candidatesNum, double fpp, 
ColumnDescriptor column) {
-this(numBytes, LOWER_BOUND_BYTES, UPPER_BOUND_BYTES, HashStrategy.XXH64, 
fpp, candidatesNum, column);
-  }
-
-  public DynamicBlockBloomFilter(int numBytes, int maximumBytes, int 
candidatesNum, double fpp, ColumnDescriptor column) {
-this(numBytes, LOWER_BOUND_BYTES, maximumBytes, HashStrategy.XXH64, fpp, 
candidatesNum, column);
+  /**
+   * Given the maximum acceptable bytes size of bloom filter, generate 
candidates according it.

Review Comment:
   ```suggestion
  * Generate bloom filter candidates according to the maximum acceptable 
byte size.
   ```



##
parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/AdaptiveBlockSplitBloomFilter.java:
##
@@ -38,57 +38,51 @@
 import org.apache.parquet.io.api.Binary;
 
 /**
- * `DynamicBlockBloomFilter` contains multiple `BlockSplitBloomFilter` as 
candidates and inserts values in
+ * `AdaptiveBlockSplitBloomFilter` contains multiple `BlockSplitBloomFilter` 
as candidates and inserts values in
  * the candidates at the same time.
  * The purpose of this is to finally generate a bloom filter with the optimal 
bit size according to the number
  * of real data distinct values. Use the largest bloom filter as an 
approximate deduplication counter, and then
  * remove incapable bloom filter candidate during data insertion.
  */
-public class DynamicBlockBloomFilter implements BloomFilter {
+public class AdaptiveBlockSplitBloomFilter implements BloomFilter {
 
-  private static final Logger LOG = 
LoggerFactory.getLogger(DynamicBlockBloomFilter.class);
+  private static final Logger LOG = 
LoggerFactory.getLogger(AdaptiveBlockSplitBloomFilter.class);
 
   // multiple candidates, inserting data at the same time. If the distinct 
values are greater than the
   // expected NDV of candidates, it will be removed. Finally we will choose 
the smallest candidate to write out.
   private final List candidates = new ArrayList<>();
 
   // the largest among candidates and used as an approximate deduplication 
counter
-  private BloomFilterCandidate maxCandidate;
+  private BloomFilterCandidate largestCandidate;
 
   // the accumulator of the number of distinct values that have been inserted 
so far
-  private int distinctValueCounter = 0;
+  private long distinctValueCounter = 0;
 
   // indicates that the bloom filter candidate has been written out and new 
data should be no longer allowed to be inserted
   private boolean finalized = false;
 
+  // indicates the step 

[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719583#comment-17719583
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

wgtmac commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535595073

   The parquet community is discussing a new `1.13.1` release to address some 
issues introduced by `1.13.0`: 
https://lists.apache.org/thread/1mjvdcmwqjcblmfkfgpd9ob2yodx7tom
   
   As the release manager, what do you think if we include this PR to the 
`1.13.1` release? @Fokko 
   




> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] wgtmac commented on pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


wgtmac commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535595073

   The parquet community is discussing a new `1.13.1` release to address some 
issues introduced by `1.13.0`: 
https://lists.apache.org/thread/1mjvdcmwqjcblmfkfgpd9ob2yodx7tom
   
   As the release manager, what do you think if we include this PR to the 
`1.13.1` release? @Fokko 
   


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719526#comment-17719526
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

clairemcginty commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535392207

   Oh no, I'm sorry I missed the boat!  Is there anything that I (or my 
organization) could do to help out with the release, @wgtmac? If it helps at 
all, having this fix released would drastically help us drive Parquet adoption, 
and lead to more OSS contributions down the line :) 




> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] clairemcginty commented on pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


clairemcginty commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535392207

   Oh no, I'm sorry I missed the boat!  Is there anything that I (or my 
organization) could do to help out with the release, @wgtmac? If it helps at 
all, having this fix released would drastically help us drive Parquet adoption, 
and lead to more OSS contributions down the line :) 


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719520#comment-17719520
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

gszadovszky commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535348289

   I wouldn't like to make you sad, @clairemcginty, but we just released 
`1.13.0` last month and the previous one was almost a year ago. Meanwhile, 
nothing says we cannot do releases for tiny features, it is more about the 
effort and who has the time to do it. I usually don't, unfortunately.
   Maybe if you ask nicely, @wgtmac would do another one... :wink:




> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] gszadovszky commented on pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


gszadovszky commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535348289

   I wouldn't like to make you sad, @clairemcginty, but we just released 
`1.13.0` last month and the previous one was almost a year ago. Meanwhile, 
nothing says we cannot do releases for tiny features, it is more about the 
effort and who has the time to do it. I usually don't, unfortunately.
   Maybe if you ask nicely, @wgtmac would do another one... :wink:


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719517#comment-17719517
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

clairemcginty commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535331312

   thanks for the review @gszadovszky! very excited to start using this in 
Parquet :) 




> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] clairemcginty commented on pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


clairemcginty commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1535331312

   thanks for the review @gszadovszky! very excited to start using this in 
Parquet :) 


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719465#comment-17719465
 ] 

ASF GitHub Bot commented on PARQUET-2276:
-

Fokko commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185348684


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -55,23 +95,31 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 
   /**
* Is the inner stream byte buffer readable?
-   * The test is "the stream is not FSDataInputStream
+   * The test is 'the stream is not FSDataInputStream
* and implements ByteBufferReadable'
*
* That is: all streams which implement ByteBufferReadable
-   * other than FSDataInputStream successfuly support read(ByteBuffer).
-   * This is true for all filesytem clients the hadoop codebase.
+   * other than FSDataInputStream successfully support read(ByteBuffer).
+   * This is true for all filesystem clients the hadoop codebase.
*
* In hadoop 3.3.0+, the StreamCapabilities probe can be used to
* check this: only those streams which provide the read(ByteBuffer)
* semantics MAY return true for the probe "in:readbytebuffer";
* FSDataInputStream will pass the probe down to the underlying stream.
*
* @param stream stream to probe
-   * @return true if it is safe to a H2SeekableInputStream to access the data
+   * @return true if it is safe to a H2SeekableInputStream to access
+   * the data, null when it cannot be determined because of missing 
hasCapabilities
*/
-  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
-if (stream.hasCapability("in:readbytebuffer")) {
+  private static Boolean 
isWrappedStreamByteBufferReadableHasCapabilities(FSDataInputStream stream) {

Review Comment:
   I agree, I've changed it to `isWrappedStreamByteBufferReadable`





> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] Fokko commented on a diff in pull request #1084: PARQUET-2276: Bring back support for Hadoop 2.7.3

2023-05-04 Thread via GitHub


Fokko commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185348684


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -55,23 +95,31 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 
   /**
* Is the inner stream byte buffer readable?
-   * The test is "the stream is not FSDataInputStream
+   * The test is 'the stream is not FSDataInputStream
* and implements ByteBufferReadable'
*
* That is: all streams which implement ByteBufferReadable
-   * other than FSDataInputStream successfuly support read(ByteBuffer).
-   * This is true for all filesytem clients the hadoop codebase.
+   * other than FSDataInputStream successfully support read(ByteBuffer).
+   * This is true for all filesystem clients the hadoop codebase.
*
* In hadoop 3.3.0+, the StreamCapabilities probe can be used to
* check this: only those streams which provide the read(ByteBuffer)
* semantics MAY return true for the probe "in:readbytebuffer";
* FSDataInputStream will pass the probe down to the underlying stream.
*
* @param stream stream to probe
-   * @return true if it is safe to a H2SeekableInputStream to access the data
+   * @return true if it is safe to a H2SeekableInputStream to access
+   * the data, null when it cannot be determined because of missing 
hasCapabilities
*/
-  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
-if (stream.hasCapability("in:readbytebuffer")) {
+  private static Boolean 
isWrappedStreamByteBufferReadableHasCapabilities(FSDataInputStream stream) {

Review Comment:
   I agree, I've changed it to `isWrappedStreamByteBufferReadable`



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719464#comment-17719464
 ] 

ASF GitHub Bot commented on PARQUET-2276:
-

Fokko commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185348113


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -46,7 +54,39 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (isWrappedStreamByteBufferReadable(stream)) {
+
+// Try to check using hasCapabilities(str)
+Boolean hasCapabilitiesResult = 
isWrappedStreamByteBufferReadableHasCapabilities(stream);
+
+// If it is null, then fall back to the old method
+if (hasCapabilitiesResult != null) {
+  if (hasCapabilitiesResult) {
+return new H2SeekableInputStream(stream);
+  } else {
+return new H1SeekableInputStream(stream);
+  }
+}
+
+return isWrappedStreamByteBufferReadableLegacy(stream);
+  }
+
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is 'the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * This logic is only used for Hadoop <2.9.x, and <3.x.x
+   *
+   * @param stream stream to probe
+   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
+   */
+  private static SeekableInputStream 
isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);

Review Comment:
   Yes, I kept it in there.





> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] Fokko commented on a diff in pull request #1084: PARQUET-2276: Bring back support for Hadoop 2.7.3

2023-05-04 Thread via GitHub


Fokko commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185348113


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -46,7 +54,39 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (isWrappedStreamByteBufferReadable(stream)) {
+
+// Try to check using hasCapabilities(str)
+Boolean hasCapabilitiesResult = 
isWrappedStreamByteBufferReadableHasCapabilities(stream);
+
+// If it is null, then fall back to the old method
+if (hasCapabilitiesResult != null) {
+  if (hasCapabilitiesResult) {
+return new H2SeekableInputStream(stream);
+  } else {
+return new H1SeekableInputStream(stream);
+  }
+}
+
+return isWrappedStreamByteBufferReadableLegacy(stream);
+  }
+
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is 'the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * This logic is only used for Hadoop <2.9.x, and <3.x.x
+   *
+   * @param stream stream to probe
+   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
+   */
+  private static SeekableInputStream 
isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);

Review Comment:
   Yes, I kept it in there.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719404#comment-17719404
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

clairemcginty commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1185224751


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -169,6 +172,46 @@ public void add(Object value) {
 }
   }
 
+  /**
+   * Returns the specific data model for a given SpecificRecord schema by 
reflecting the underlying
+   * Avro class's `MODEL$` field, or Null if the class is not on the classpath 
or reflection fails.
+   */
+  static SpecificData getModelForSchema(Schema schema) {
+final Class clazz;
+
+if (schema != null && (schema.getType() == Schema.Type.RECORD || 
schema.getType() == Schema.Type.UNION)) {
+  clazz = SpecificData.get().getClass(schema);
+} else {
+  return null;
+}
+
+final SpecificData model;
+try {
+  final Field modelField = clazz.getDeclaredField("MODEL$");
+  modelField.setAccessible(true);
+
+  model = (SpecificData) modelField.get(null);
+} catch (Exception e) {
+  return null;
+}
+
+try {
+  final String avroVersion = 
Schema.Parser.class.getPackage().getImplementationVersion();
+  // Avro 1.8 doesn't include conversions in the MODEL$ field
+  if (avroVersion.startsWith("1.8.")) {

Review Comment:
   Great! I added code wrapping `getModelForSchema` in a try/catch at each call 
site.





> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] clairemcginty commented on a diff in pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


clairemcginty commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1185224751


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -169,6 +172,46 @@ public void add(Object value) {
 }
   }
 
+  /**
+   * Returns the specific data model for a given SpecificRecord schema by 
reflecting the underlying
+   * Avro class's `MODEL$` field, or Null if the class is not on the classpath 
or reflection fails.
+   */
+  static SpecificData getModelForSchema(Schema schema) {
+final Class clazz;
+
+if (schema != null && (schema.getType() == Schema.Type.RECORD || 
schema.getType() == Schema.Type.UNION)) {
+  clazz = SpecificData.get().getClass(schema);
+} else {
+  return null;
+}
+
+final SpecificData model;
+try {
+  final Field modelField = clazz.getDeclaredField("MODEL$");
+  modelField.setAccessible(true);
+
+  model = (SpecificData) modelField.get(null);
+} catch (Exception e) {
+  return null;
+}
+
+try {
+  final String avroVersion = 
Schema.Parser.class.getPackage().getImplementationVersion();
+  // Avro 1.8 doesn't include conversions in the MODEL$ field
+  if (avroVersion.startsWith("1.8.")) {

Review Comment:
   Great! I added code wrapping `getModelForSchema` in a try/catch at each call 
site.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719388#comment-17719388
 ] 

ASF GitHub Bot commented on PARQUET-2276:
-

shangxinli commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185181599


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -55,23 +95,31 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 
   /**
* Is the inner stream byte buffer readable?
-   * The test is "the stream is not FSDataInputStream
+   * The test is 'the stream is not FSDataInputStream
* and implements ByteBufferReadable'
*
* That is: all streams which implement ByteBufferReadable
-   * other than FSDataInputStream successfuly support read(ByteBuffer).
-   * This is true for all filesytem clients the hadoop codebase.
+   * other than FSDataInputStream successfully support read(ByteBuffer).
+   * This is true for all filesystem clients the hadoop codebase.
*
* In hadoop 3.3.0+, the StreamCapabilities probe can be used to
* check this: only those streams which provide the read(ByteBuffer)
* semantics MAY return true for the probe "in:readbytebuffer";
* FSDataInputStream will pass the probe down to the underlying stream.
*
* @param stream stream to probe
-   * @return true if it is safe to a H2SeekableInputStream to access the data
+   * @return true if it is safe to a H2SeekableInputStream to access
+   * the data, null when it cannot be determined because of missing 
hasCapabilities
*/
-  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
-if (stream.hasCapability("in:readbytebuffer")) {
+  private static Boolean 
isWrappedStreamByteBufferReadableHasCapabilities(FSDataInputStream stream) {

Review Comment:
   This long method name is more meaningful but it seems too long. But this is 
a minor comment and doesn't need to be changed if no better name. 





> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1084: PARQUET-2276: Bring back support for Hadoop 2.7.3

2023-05-04 Thread via GitHub


shangxinli commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185181599


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -55,23 +95,31 @@ public static SeekableInputStream wrap(FSDataInputStream 
stream) {
 
   /**
* Is the inner stream byte buffer readable?
-   * The test is "the stream is not FSDataInputStream
+   * The test is 'the stream is not FSDataInputStream
* and implements ByteBufferReadable'
*
* That is: all streams which implement ByteBufferReadable
-   * other than FSDataInputStream successfuly support read(ByteBuffer).
-   * This is true for all filesytem clients the hadoop codebase.
+   * other than FSDataInputStream successfully support read(ByteBuffer).
+   * This is true for all filesystem clients the hadoop codebase.
*
* In hadoop 3.3.0+, the StreamCapabilities probe can be used to
* check this: only those streams which provide the read(ByteBuffer)
* semantics MAY return true for the probe "in:readbytebuffer";
* FSDataInputStream will pass the probe down to the underlying stream.
*
* @param stream stream to probe
-   * @return true if it is safe to a H2SeekableInputStream to access the data
+   * @return true if it is safe to a H2SeekableInputStream to access
+   * the data, null when it cannot be determined because of missing 
hasCapabilities
*/
-  private static boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
-if (stream.hasCapability("in:readbytebuffer")) {
+  private static Boolean 
isWrappedStreamByteBufferReadableHasCapabilities(FSDataInputStream stream) {

Review Comment:
   This long method name is more meaningful but it seems too long. But this is 
a minor comment and doesn't need to be changed if no better name. 



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719387#comment-17719387
 ] 

ASF GitHub Bot commented on PARQUET-2276:
-

shangxinli commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185179904


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -46,7 +54,39 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (isWrappedStreamByteBufferReadable(stream)) {
+
+// Try to check using hasCapabilities(str)
+Boolean hasCapabilitiesResult = 
isWrappedStreamByteBufferReadableHasCapabilities(stream);
+
+// If it is null, then fall back to the old method
+if (hasCapabilitiesResult != null) {
+  if (hasCapabilitiesResult) {
+return new H2SeekableInputStream(stream);
+  } else {
+return new H1SeekableInputStream(stream);
+  }
+}
+
+return isWrappedStreamByteBufferReadableLegacy(stream);
+  }
+
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is 'the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * This logic is only used for Hadoop <2.9.x, and <3.x.x
+   *
+   * @param stream stream to probe
+   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
+   */
+  private static SeekableInputStream 
isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);

Review Comment:
   It is good to add this debug log in case recursive methods cause an infinite 
loop, we can enable this logging and debug. 





> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] shangxinli commented on a diff in pull request #1084: PARQUET-2276: Bring back support for Hadoop 2.7.3

2023-05-04 Thread via GitHub


shangxinli commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1185179904


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -46,7 +54,39 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (isWrappedStreamByteBufferReadable(stream)) {
+
+// Try to check using hasCapabilities(str)
+Boolean hasCapabilitiesResult = 
isWrappedStreamByteBufferReadableHasCapabilities(stream);
+
+// If it is null, then fall back to the old method
+if (hasCapabilitiesResult != null) {
+  if (hasCapabilitiesResult) {
+return new H2SeekableInputStream(stream);
+  } else {
+return new H1SeekableInputStream(stream);
+  }
+}
+
+return isWrappedStreamByteBufferReadableLegacy(stream);
+  }
+
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is 'the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * This logic is only used for Hadoop <2.9.x, and <3.x.x
+   *
+   * @param stream stream to probe
+   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
+   */
+  private static SeekableInputStream 
isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);

Review Comment:
   It is good to add this debug log in case recursive methods cause an infinite 
loop, we can enable this logging and debug. 



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719366#comment-17719366
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

gszadovszky commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1185124781


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -169,6 +172,46 @@ public void add(Object value) {
 }
   }
 
+  /**
+   * Returns the specific data model for a given SpecificRecord schema by 
reflecting the underlying
+   * Avro class's `MODEL$` field, or Null if the class is not on the classpath 
or reflection fails.
+   */
+  static SpecificData getModelForSchema(Schema schema) {
+final Class clazz;
+
+if (schema != null && (schema.getType() == Schema.Type.RECORD || 
schema.getType() == Schema.Type.UNION)) {
+  clazz = SpecificData.get().getClass(schema);
+} else {
+  return null;
+}
+
+final SpecificData model;
+try {
+  final Field modelField = clazz.getDeclaredField("MODEL$");
+  modelField.setAccessible(true);
+
+  model = (SpecificData) modelField.get(null);
+} catch (Exception e) {
+  return null;
+}
+
+try {
+  final String avroVersion = 
Schema.Parser.class.getPackage().getImplementationVersion();
+  // Avro 1.8 doesn't include conversions in the MODEL$ field
+  if (avroVersion.startsWith("1.8.")) {

Review Comment:
   > so I've tested 1.7 and 1.8; since 1.9 Avro has stably used the `MODEL$` 
field to hold all conversions, so I feel reasonably confident about relying on 
this. If that changes, we'll catch it in the new unit tests +1
   
   This sounds perfect to me. Thanks a lot for the additional work!
   
   > If you want, I can surround invocations of `getModelForSchema` in a 
try/catch (in `AvroReadSupport`/`AvroWriteSupport`), and just use the default 
SpecificDataSupplier if they throw anything. That way any unexpected behavior 
would just result in logical types not being used.
   
   Yes, I think this fallback mechanism sounds reasonable to me.
   
   





> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719365#comment-17719365
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

gszadovszky commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1185124781


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -169,6 +172,46 @@ public void add(Object value) {
 }
   }
 
+  /**
+   * Returns the specific data model for a given SpecificRecord schema by 
reflecting the underlying
+   * Avro class's `MODEL$` field, or Null if the class is not on the classpath 
or reflection fails.
+   */
+  static SpecificData getModelForSchema(Schema schema) {
+final Class clazz;
+
+if (schema != null && (schema.getType() == Schema.Type.RECORD || 
schema.getType() == Schema.Type.UNION)) {
+  clazz = SpecificData.get().getClass(schema);
+} else {
+  return null;
+}
+
+final SpecificData model;
+try {
+  final Field modelField = clazz.getDeclaredField("MODEL$");
+  modelField.setAccessible(true);
+
+  model = (SpecificData) modelField.get(null);
+} catch (Exception e) {
+  return null;
+}
+
+try {
+  final String avroVersion = 
Schema.Parser.class.getPackage().getImplementationVersion();
+  // Avro 1.8 doesn't include conversions in the MODEL$ field
+  if (avroVersion.startsWith("1.8.")) {

Review Comment:
   > so I've tested 1.7 and 1.8; since 1.9 Avro has stably used the `MODEL$` 
field to hold all conversions, so I feel reasonably confident about relying on 
this. If that changes, we'll catch it in the new unit tests +1
   This sounds perfect to me. Thanks a lot for the additional work!
   
   > If you want, I can surround invocations of `getModelForSchema` in a 
try/catch (in `AvroReadSupport`/`AvroWriteSupport`), and just use the default 
SpecificDataSupplier if they throw anything. That way any unexpected behavior 
would just result in logical types not being used.
   Yes, I think this fallback mechanism sounds reasonable to me.
   
   





> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


gszadovszky commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1185124781


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -169,6 +172,46 @@ public void add(Object value) {
 }
   }
 
+  /**
+   * Returns the specific data model for a given SpecificRecord schema by 
reflecting the underlying
+   * Avro class's `MODEL$` field, or Null if the class is not on the classpath 
or reflection fails.
+   */
+  static SpecificData getModelForSchema(Schema schema) {
+final Class clazz;
+
+if (schema != null && (schema.getType() == Schema.Type.RECORD || 
schema.getType() == Schema.Type.UNION)) {
+  clazz = SpecificData.get().getClass(schema);
+} else {
+  return null;
+}
+
+final SpecificData model;
+try {
+  final Field modelField = clazz.getDeclaredField("MODEL$");
+  modelField.setAccessible(true);
+
+  model = (SpecificData) modelField.get(null);
+} catch (Exception e) {
+  return null;
+}
+
+try {
+  final String avroVersion = 
Schema.Parser.class.getPackage().getImplementationVersion();
+  // Avro 1.8 doesn't include conversions in the MODEL$ field
+  if (avroVersion.startsWith("1.8.")) {

Review Comment:
   > so I've tested 1.7 and 1.8; since 1.9 Avro has stably used the `MODEL$` 
field to hold all conversions, so I feel reasonably confident about relying on 
this. If that changes, we'll catch it in the new unit tests +1
   
   This sounds perfect to me. Thanks a lot for the additional work!
   
   > If you want, I can surround invocations of `getModelForSchema` in a 
try/catch (in `AvroReadSupport`/`AvroWriteSupport`), and just use the default 
SpecificDataSupplier if they throw anything. That way any unexpected behavior 
would just result in logical types not being used.
   
   Yes, I think this fallback mechanism sounds reasonable to me.
   
   



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] gszadovszky commented on a diff in pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


gszadovszky commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1185124781


##
parquet-avro/src/main/java/org/apache/parquet/avro/AvroRecordConverter.java:
##
@@ -169,6 +172,46 @@ public void add(Object value) {
 }
   }
 
+  /**
+   * Returns the specific data model for a given SpecificRecord schema by 
reflecting the underlying
+   * Avro class's `MODEL$` field, or Null if the class is not on the classpath 
or reflection fails.
+   */
+  static SpecificData getModelForSchema(Schema schema) {
+final Class clazz;
+
+if (schema != null && (schema.getType() == Schema.Type.RECORD || 
schema.getType() == Schema.Type.UNION)) {
+  clazz = SpecificData.get().getClass(schema);
+} else {
+  return null;
+}
+
+final SpecificData model;
+try {
+  final Field modelField = clazz.getDeclaredField("MODEL$");
+  modelField.setAccessible(true);
+
+  model = (SpecificData) modelField.get(null);
+} catch (Exception e) {
+  return null;
+}
+
+try {
+  final String avroVersion = 
Schema.Parser.class.getPackage().getImplementationVersion();
+  // Avro 1.8 doesn't include conversions in the MODEL$ field
+  if (avroVersion.startsWith("1.8.")) {

Review Comment:
   > so I've tested 1.7 and 1.8; since 1.9 Avro has stably used the `MODEL$` 
field to hold all conversions, so I feel reasonably confident about relying on 
this. If that changes, we'll catch it in the new unit tests +1
   This sounds perfect to me. Thanks a lot for the additional work!
   
   > If you want, I can surround invocations of `getModelForSchema` in a 
try/catch (in `AvroReadSupport`/`AvroWriteSupport`), and just use the default 
SpecificDataSupplier if they throw anything. That way any unexpected behavior 
would just result in logical types not being used.
   Yes, I think this fallback mechanism sounds reasonable to me.
   
   



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719335#comment-17719335
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

clairemcginty commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1534819205

   @wgtmac, sorry about that 

> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] clairemcginty commented on pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


clairemcginty commented on PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#issuecomment-1534819205

   @wgtmac, sorry about that -- `mvn verify` was happy on my machine, but I 
wasn't running `install`. Should be good now!


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2292) Improve default SpecificRecord model selection for Avro{Write,Read}Support

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2292?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719317#comment-17719317
 ] 

ASF GitHub Bot commented on PARQUET-2292:
-

clairemcginty commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1184991179


##
parquet-avro/pom.xml:
##
@@ -105,6 +110,30 @@
   test-jar
   test
 
+
+  org.mockito
+  mockito-core
+  2.23.0

Review Comment:
   It should be, but it does require an artifact migration in a few modules 
(parquet-column and parquet-hadoop depend on mockito-all, which has moved to 
mockito-core in 2.x). I would be happy to do it, just not sure if it's in scope 
for this PR :) 





> Improve default SpecificRecord model selection for Avro{Write,Read}Support
> --
>
> Key: PARQUET-2292
> URL: https://issues.apache.org/jira/browse/PARQUET-2292
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Claire McGinty
>Assignee: Claire McGinty
>Priority: Major
>
> AvroWriteSupport/AvroReadSupport can improve the precision of their default 
> `model` selection. Currently they default to new 
> SpecificDataSupplier().get()[0]. This means that SpecificRecord classes that 
> contain logical types will fail out-of-the-box unless a specific 
> DATA_SUPPLIER is configured that contains logical type conversions.
> I think we can improve this and make logical types work by default by 
> defaulting to the value of the `MODEL$` field that every SpecificRecordBase 
> implementation contains, which already contains all the logical conversions 
> for that Avro type. It would require reflection, but that's what the Avro 
> library is already doing to fetch models for Specific types[1].
>  
> [0] 
> [https://github.com/apache/parquet-mr/blob/d38044f5395494e1543581a4b763f624305d3022/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L403-L407]
> [1] 
> https://github.com/apache/avro/blob/release-1.11.1/lang/java/avro/src/main/java/org/apache/avro/specific/SpecificData.java#L76-L86



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] clairemcginty commented on a diff in pull request #1078: PARQUET-2292: Default SpecificRecord model reflects from MODEL$ field

2023-05-04 Thread via GitHub


clairemcginty commented on code in PR #1078:
URL: https://github.com/apache/parquet-mr/pull/1078#discussion_r1184991179


##
parquet-avro/pom.xml:
##
@@ -105,6 +110,30 @@
   test-jar
   test
 
+
+  org.mockito
+  mockito-core
+  2.23.0

Review Comment:
   It should be, but it does require an artifact migration in a few modules 
(parquet-column and parquet-hadoop depend on mockito-all, which has moved to 
mockito-core in 2.x). I would be happy to do it, just not sure if it's in scope 
for this PR :) 



-- 
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: dev-unsubscr...@parquet.apache.org

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



[GitHub] [parquet-mr] Fokko merged pull request #1085: Bump actions/setup-java from 1 to 3

2023-05-04 Thread via GitHub


Fokko merged PR #1085:
URL: https://github.com/apache/parquet-mr/pull/1085


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719302#comment-17719302
 ] 

ASF GitHub Bot commented on PARQUET-2276:
-

steveloughran commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1184944471


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -46,41 +83,91 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (isWrappedStreamByteBufferReadable(stream)) {
-  return new H2SeekableInputStream(stream);
-} else {
-  return new H1SeekableInputStream(stream);
+
+// Try to check using hasCapabilities(str)
+Boolean hasCapabilitiesResult = 
isWrappedStreamByteBufferReadableHasCapabilities(stream);
+
+// If it is null, then fall back to the old method
+if (hasCapabilitiesResult != null) {
+  if (hasCapabilitiesResult) {
+return new H2SeekableInputStream(stream);
+  } else {
+return new H1SeekableInputStream(stream);
+  }
+}
+
+return isWrappedStreamByteBufferReadableLegacy(stream);
+  }
+
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is 'the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * This logic is only used for Hadoop <2.9.x, and <3.x.x
+   *
+   * @param stream stream to probe
+   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
+   */
+  private static SeekableInputStream 
isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);
+  return isWrappedStreamByteBufferReadableLegacy(((FSDataInputStream) 
wrapped));

Review Comment:
   you can if you try hard, it's just really unusual
   
   you can never wrap an instance by itself.





> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #1084: PARQUET-2276: Bring back support for Hadoop 2.7.3

2023-05-04 Thread via GitHub


steveloughran commented on code in PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#discussion_r1184944471


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##
@@ -46,41 +83,91 @@ public class HadoopStreams {
*/
   public static SeekableInputStream wrap(FSDataInputStream stream) {
 Objects.requireNonNull(stream, "Cannot wrap a null input stream");
-if (isWrappedStreamByteBufferReadable(stream)) {
-  return new H2SeekableInputStream(stream);
-} else {
-  return new H1SeekableInputStream(stream);
+
+// Try to check using hasCapabilities(str)
+Boolean hasCapabilitiesResult = 
isWrappedStreamByteBufferReadableHasCapabilities(stream);
+
+// If it is null, then fall back to the old method
+if (hasCapabilitiesResult != null) {
+  if (hasCapabilitiesResult) {
+return new H2SeekableInputStream(stream);
+  } else {
+return new H1SeekableInputStream(stream);
+  }
+}
+
+return isWrappedStreamByteBufferReadableLegacy(stream);
+  }
+
+  /**
+   * Is the inner stream byte buffer readable?
+   * The test is 'the stream is not FSDataInputStream
+   * and implements ByteBufferReadable'
+   *
+   * This logic is only used for Hadoop <2.9.x, and <3.x.x
+   *
+   * @param stream stream to probe
+   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
+   */
+  private static SeekableInputStream 
isWrappedStreamByteBufferReadableLegacy(FSDataInputStream stream) {
+InputStream wrapped = stream.getWrappedStream();
+if (wrapped instanceof FSDataInputStream) {
+  LOG.debug("Checking on wrapped stream {} of {} whether is 
ByteBufferReadable", wrapped, stream);
+  return isWrappedStreamByteBufferReadableLegacy(((FSDataInputStream) 
wrapped));

Review Comment:
   you can if you try hard, it's just really unusual
   
   you can never wrap an instance by itself.



-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2276) ParquetReader reads do not work with Hadoop version 2.8.5

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719298#comment-17719298
 ] 

ASF GitHub Bot commented on PARQUET-2276:
-

steveloughran commented on PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#issuecomment-1534680365

   I repeat my stance on this: to claim hadoop 2.7 runtime compatibility you 
should be building against java7. if you don't, well, make clear its a fairly 
qualified support "hadoop 2.7.3 on java8 only" and not worry about the bits of 
hadoop which break if they try to do that (kerberos, s3a, anything with 
joda-time, ...)




> ParquetReader reads do not work with Hadoop version 2.8.5
> -
>
> Key: PARQUET-2276
> URL: https://issues.apache.org/jira/browse/PARQUET-2276
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Atul Mohan
>Assignee: Fokko Driesprong
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> {{ParquetReader.read() fails with the following exception on parquet-mr 
> version 1.13.0 when using hadoop version 2.8.5:}}
> {code:java}
>  java.lang.NoSuchMethodError: 'boolean 
> org.apache.hadoop.fs.FSDataInputStream.hasCapability(java.lang.String)' 
> at 
> org.apache.parquet.hadoop.util.HadoopStreams.isWrappedStreamByteBufferReadable(HadoopStreams.java:74)
>  
> at org.apache.parquet.hadoop.util.HadoopStreams.wrap(HadoopStreams.java:49) 
> at 
> org.apache.parquet.hadoop.util.HadoopInputFile.newStream(HadoopInputFile.java:69)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.(ParquetFileReader.java:787)
>  
> at 
> org.apache.parquet.hadoop.ParquetFileReader.open(ParquetFileReader.java:657) 
> at org.apache.parquet.hadoop.ParquetReader.initReader(ParquetReader.java:162) 
> org.apache.parquet.hadoop.ParquetReader.read(ParquetReader.java:135)
> {code}
>  
>  
>  
> From an initial investigation, it looks like HadoopStreams has started using 
> [FSDataInputStream.hasCapability|https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java#L74]
>  but _FSDataInputStream_ does not have the _hasCapability_ API in [hadoop 
> 2.8.x|https://hadoop.apache.org/docs/r2.8.3/api/org/apache/hadoop/fs/FSDataInputStream.html].



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] steveloughran commented on pull request #1084: PARQUET-2276: Bring back support for Hadoop 2.7.3

2023-05-04 Thread via GitHub


steveloughran commented on PR #1084:
URL: https://github.com/apache/parquet-mr/pull/1084#issuecomment-1534680365

   I repeat my stance on this: to claim hadoop 2.7 runtime compatibility you 
should be building against java7. if you don't, well, make clear its a fairly 
qualified support "hadoop 2.7.3 on java8 only" and not worry about the bits of 
hadoop which break if they try to do that (kerberos, s3a, anything with 
joda-time, ...)


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Commented] (PARQUET-2193) Encrypting only one field in nested field prevents reading of other fields in nested field without keys

2023-05-04 Thread Gidon Gershinsky (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719294#comment-17719294
 ] 

Gidon Gershinsky commented on PARQUET-2193:
---

[~Nageswaran] A couple of updates on this.

We should be able to skip this verification for encrypted files, a pull request 
is sent to parquet-mr.

Also, I've tried the new Spark 3.4.0 (as is, no modifications) with the scala 
test above - no exception was thrown. Probably, the updated Spark code bypasses 
the problematic parquet read path. Can you check if Spark 3.4.0 works ok for 
your usecase.

> Encrypting only one field in nested field prevents reading of other fields in 
> nested field without keys
> ---
>
> Key: PARQUET-2193
> URL: https://issues.apache.org/jira/browse/PARQUET-2193
> Project: Parquet
>  Issue Type: New Feature
>  Components: parquet-mr
>Affects Versions: 1.12.0
>Reporter: Vignesh Nageswaran
>Priority: Major
>
> Hi Team,
> While exploring parquet encryption, it is found that, if a field in nested 
> column is encrypted , and If I want to read this parquet directory from other 
> applications which does not have encryption keys to decrypt it, I cannot read 
> the remaining fields of the nested column without keys. 
> Example 
> `
> {code:java}
> case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0)
> case class SquareItem(int_column: Int, square_int_column : Double, 
> partitionCol: Int, nestedCol :nestedItem)
> `{code}
> In the case class `SquareItem` , `nestedCol` field is nested field and I want 
> to encrypt a field `ic` within it. 
>  
> I also want the footer to be non encrypted , so that I can use the encrypted 
> parquet file by legacy applications. 
>  
> Encryption is successful, however, when I query the parquet file using spark 
> 3.3.0 without having any configuration for parquet encryption set up , I 
> cannot non encrypted fields of `nestedCol` `sic`. I was expecting that only 
> `nestedCol` `ic` field will not be querable.
>  
>  
> Reproducer. 
> Spark 3.3.0 Using Spark-shell 
> Downloaded the file 
> [parquet-hadoop-1.12.0-tests.jar|https://repo1.maven.org/maven2/org/apache/parquet/parquet-hadoop/1.12.0/parquet-hadoop-1.12.0-tests.jar]
>  and added it to spark-jars folder
> Code to create encrypted data. #  
>  
> {code:java}
> sc.hadoopConfiguration.set("parquet.crypto.factory.class" 
> ,"org.apache.parquet.crypto.keytools.PropertiesDrivenCryptoFactory")
> sc.hadoopConfiguration.set("parquet.encryption.kms.client.class" 
> ,"org.apache.parquet.crypto.keytools.mocks.InMemoryKMS")
> sc.hadoopConfiguration.set("parquet.encryption.key.list","key1a: 
> BAECAwQFBgcICQoLDA0ODw==, key2a: BAECAAECAAECAAECAAECAA==, keyz: 
> BAECAAECAAECAAECAAECAA==")
> sc.hadoopConfiguration.set("parquet.encryption.key.material.store.internally","false")
> val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted"
> valpartitionCol = 1
> case class nestedItem(ic: Int = 0, sic : Double, pc: Int = 0)
> case class SquareItem(int_column: Int, square_int_column : Double, 
> partitionCol: Int, nestedCol :nestedItem)
> val dataRange = (1 to 100).toList
> val squares = sc.parallelize(dataRange.map(i => new SquareItem(i, 
> scala.math.pow(i,2), partitionCol,nestedItem(i,i
> squares.toDS().show()
> squares.toDS().write.partitionBy("partitionCol").mode("overwrite").option("parquet.encryption.column.keys",
>  
> "key1a:square_int_column,nestedCol.ic;").option("parquet.encryption.plaintext.footer",true).option("parquet.encryption.footer.key",
>  "keyz").parquet(encryptedParquetPath)
> {code}
> Code to read the data trying to access non encrypted nested field by opening 
> a new spark-shell
>  
> {code:java}
> val encryptedParquetPath = "/tmp/par_enc_footer_non_encrypted"
> spark.sqlContext.read.parquet(encryptedParquetPath).createOrReplaceTempView("test")
> spark.sql("select nestedCol.sic from test").show(){code}
> As you can see that nestedCol.sic is not encrypted , I was expecting the 
> results, but
> I get the below error
>  
> {code:java}
> Caused by: org.apache.parquet.crypto.ParquetCryptoRuntimeException: 
> [square_int_column]. Null File Decryptor
>   at 
> org.apache.parquet.hadoop.metadata.EncryptedColumnChunkMetaData.decryptIfNeeded(ColumnChunkMetaData.java:602)
>   at 
> org.apache.parquet.hadoop.metadata.ColumnChunkMetaData.getEncodings(ColumnChunkMetaData.java:348)
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.checkDeltaByteArrayProblem(ParquetRecordReader.java:191)
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:177)
>   at 
> org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
>   at 
> 

[jira] [Commented] (PARQUET-2297) Encrypted files should not be checked for delta encoding problem

2023-05-04 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2297?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17719170#comment-17719170
 ] 

ASF GitHub Bot commented on PARQUET-2297:
-

ggershinsky opened a new pull request, #1089:
URL: https://github.com/apache/parquet-mr/pull/1089

   https://issues.apache.org/jira/browse/PARQUET-2297




> Encrypted files should not be checked for delta encoding problem
> 
>
> Key: PARQUET-2297
> URL: https://issues.apache.org/jira/browse/PARQUET-2297
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.13.0
>Reporter: Gidon Gershinsky
>Assignee: Gidon Gershinsky
>Priority: Major
> Fix For: 1.14.0, 1.13.1
>
>
> Delta encoding problem (https://issues.apache.org/jira/browse/PARQUET-246) 
> was fixed in writers since parquet-mr-1.8. This fix also added a 
> `checkDeltaByteArrayProblem` method in readers, that runs over all columns 
> and checks for this problem in older files. 
> This now triggers an unrelated exception when reading encrypted files, in the 
> following situation: trying to read an unencrypted column, without having 
> keys for encrypted columns (see 
> https://issues.apache.org/jira/browse/PARQUET-2193). This happens in Spark, 
> with nested columns (files with regular columns are ok).
> Possible solution: don't call the `checkDeltaByteArrayProblem` method for 
> encrypted files - because these files can be written only with 
> parquet-mr-1.12 and newer, where the delta encoding problem is already fixed.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] ggershinsky opened a new pull request, #1089: PARQUET-2297: Skip delta problem check for encrypted files

2023-05-04 Thread via GitHub


ggershinsky opened a new pull request, #1089:
URL: https://github.com/apache/parquet-mr/pull/1089

   https://issues.apache.org/jira/browse/PARQUET-2297


-- 
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: dev-unsubscr...@parquet.apache.org

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



[jira] [Created] (PARQUET-2297) Encrypted files should not be checked for delta encoding problem

2023-05-04 Thread Gidon Gershinsky (Jira)
Gidon Gershinsky created PARQUET-2297:
-

 Summary: Encrypted files should not be checked for delta encoding 
problem
 Key: PARQUET-2297
 URL: https://issues.apache.org/jira/browse/PARQUET-2297
 Project: Parquet
  Issue Type: Improvement
  Components: parquet-mr
Affects Versions: 1.13.0
Reporter: Gidon Gershinsky
Assignee: Gidon Gershinsky
 Fix For: 1.14.0, 1.13.1


Delta encoding problem (https://issues.apache.org/jira/browse/PARQUET-246) was 
fixed in writers since parquet-mr-1.8. This fix also added a 
`checkDeltaByteArrayProblem` method in readers, that runs over all columns and 
checks for this problem in older files. 

This now triggers an unrelated exception when reading encrypted files, in the 
following situation: trying to read an unencrypted column, without having keys 
for encrypted columns (see https://issues.apache.org/jira/browse/PARQUET-2193). 
This happens in Spark, with nested columns (files with regular columns are ok).

Possible solution: don't call the `checkDeltaByteArrayProblem` method for 
encrypted files - because these files can be written only with parquet-mr-1.12 
and newer, where the delta encoding problem is already fixed.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)