sqoop-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Fero Szabo via Review Board <nore...@reviews.apache.org>
Subject Re: Review Request 69060: SQOOP-3382 Add parquet numeric support for Parquet in hdfs import
Date Wed, 24 Oct 2018 15:59:35 GMT


> On Oct. 24, 2018, 2:31 p.m., Szabolcs Vasas wrote:
> > Hi Feró,
> > 
> > Thank you for submitting this improvement!
> > I have left some comments, see them below.
> > Apart from that I think we need to test explicitly that if the sqoop.parquet.logical_types.decimal.enable
flag is true then the Parquet file contains a decimal value and otherwise it contains a string
value.
> > 
> > NumericTypesImportTest asserts on string values so it is not able to verify this,
most of the tests passed even if I commented out the content of the addEnableParquetDecimal
method.

I'll look into this one.

I'm thinking that using the org.apache.sqoop.util.ParquetReader#readAll method could help
(since it returns GenericRecords), though I'm not sure. I'll somehow need to actually turn
off the conversion and check for the bytes. Any suggestions?


> On Oct. 24, 2018, 2:31 p.m., Szabolcs Vasas wrote:
> > src/java/org/apache/sqoop/mapreduce/ImportJobBase.java
> > Lines 115-119 (patched)
> > <https://reviews.apache.org/r/69060/diff/2/?file=2099869#file2099869line115>
> >
> >     Is it possible to move this to org.apache.sqoop.mapreduce.parquet.hadoop.HadoopParquetImportJobConfigurator#configureMapper?
> >     That would be consistent with the way we configure the Parquet imports but I
am not sure the effect would remain the same.

Yes, makes sense.


> On Oct. 24, 2018, 2:31 p.m., Szabolcs Vasas wrote:
> > src/java/org/apache/sqoop/mapreduce/ParquetImportMapper.java
> > Lines 56 (patched)
> > <https://reviews.apache.org/r/69060/diff/2/?file=2099870#file2099870line56>
> >
> >     Are we sure that adding the logical type conversion only here is enough?
> >     In case of Avro it is also added in org.apache.sqoop.mapreduce.AvroOutputFormat#getRecordWriter
which gets invoked in every mapper so I assume that we have to add the conversion in every
mapper in case of Parquet files too.

My understanding is that this method is invoked in every mapper. (it's doc suggest this as
well: "Called once at the beginning of the task.")
Where else would put this statement?


- Fero


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/69060/#review209954
-----------------------------------------------------------


On Oct. 24, 2018, 12:25 p.m., Fero Szabo wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/69060/
> -----------------------------------------------------------
> 
> (Updated Oct. 24, 2018, 12:25 p.m.)
> 
> 
> Review request for Sqoop, Boglarka Egyed and Szabolcs Vasas.
> 
> 
> Bugs: SQOOP-3382
>     https://issues.apache.org/jira/browse/SQOOP-3382
> 
> 
> Repository: sqoop-trunk
> 
> 
> Description
> -------
> 
> This patch is about adding support for fixed point decimal types in parquet import.
> 
> The implementation is simple after the fact that parquet was upgraded to 1.9.0 in SQOOP-3381:
we just need to register the GenericDataSupplier with AvroParquetOutputFormat.
> 
> For testing, we can reuse the existing Avro tests, because Sqoop uses Avro under the
hood to write parquet.
> 
> I also moved around and renamed the classes involved in this change so their name and
package reflect their purpose.
> 
> ** Note: A key design decision can be seen in the ImportJobTestConfiguration interface
**
> - I decided to create a new function to get the expected results for each file format,
since we seldom add new fileformats. 
> - However this also enforces future configurations to always define their expected result
for every file forma or throw a NotImplementedException should they lack the support for one.
> - The alternative for this is to define the fileLayout as an input parameter instead.
This would allow for better extendability.
> _Please share your thoughts on this!_
> 
> 
> Diffs
> -----
> 
>   src/java/org/apache/sqoop/config/ConfigurationConstants.java 3724f250 
>   src/java/org/apache/sqoop/mapreduce/ImportJobBase.java 80c06988 
>   src/java/org/apache/sqoop/mapreduce/ParquetImportMapper.java 62334f8a 
>   src/java/org/apache/sqoop/orm/AvroSchemaGenerator.java 7a2a5f9c 
>   src/test/org/apache/sqoop/importjob/ImportJobTestConfiguration.java 14de910b 
>   src/test/org/apache/sqoop/importjob/avro/AvroImportForNumericTypesTest.java ff13dc3b

>   src/test/org/apache/sqoop/importjob/avro/configuration/MSSQLServerImportJobTestConfiguration.java
182d2967 
>   src/test/org/apache/sqoop/importjob/avro/configuration/MySQLImportJobTestConfiguration.java
e9bf9912 
>   src/test/org/apache/sqoop/importjob/avro/configuration/OracleImportJobTestConfiguration.java
b7bad08c 
>   src/test/org/apache/sqoop/importjob/avro/configuration/OracleImportJobTestConfigurationForNumber.java
465e61f4 
>   src/test/org/apache/sqoop/importjob/avro/configuration/PostgresqlImportJobTestConfigurationForNumeric.java
66715c17 
>   src/test/org/apache/sqoop/importjob/avro/configuration/PostgresqlImportJobTestConfigurationPaddingShouldSucceed.java
ec4db41b 
>   src/test/org/apache/sqoop/util/ParquetReader.java 908ce566 
> 
> 
> Diff: https://reviews.apache.org/r/69060/diff/2/
> 
> 
> Testing
> -------
> 
> 3rd party tests and unit tests, both gradle and ant
> 
> 
> Thanks,
> 
> Fero Szabo
> 
>


Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message