a-agmon opened a new issue, #5593:
URL: https://github.com/apache/iceberg/issues/5593
### Apache Iceberg version
0.14.0 (latest release)
### Query engine
Spark
### Please describe the bug 🐞
**Description**
1. An Iceberg table has been created and populated in a Spark job (using
GlueCatalog)
2. Additional data has been inserted to the table using Athena console or API
3. When I attempt to read new data from the table using Spark the following
exception is thrown
```
22/08/20 22:16:05 ERROR BaseDataReader: Error reading file:
s3://****/temp-testing-aa/default.db/iceberg_demo3/data/category=CAT5/d53b94be-8d2d-4892-b020-b60610a30318.gz.parquet
java.lang.UnsupportedOperationException: Cannot support vectorized reads for
column [email] optional binary email (STRING) = 1 with encoding
DELTA_BYTE_ARRAY. Disable vectorized reads to read this table/file
at
org.apache.iceberg.arrow.vectorized.parquet.VectorizedPageIterator.initDataReader(VectorizedPageIterator.java:96)
```
This does not happen when the same data is inserted using Spark
**Reproduce**:
run this spark code
```
val spark = SparkSession.builder()
.appName("DataExtractor")
.config("spark.master", "local[*]")
.config("spark.driver.host", "127.0.0.1")
.config("spark.driver.bindAddress", "127.0.0.1")
.config("spark.hadoop.fs.s3.impl",
"org.apache.hadoop.fs.s3a.S3AFileSystem")
.config("spark.sql.catalog.my_catalog",
"org.apache.iceberg.spark.SparkCatalog")
.config("spark.sql.catalog.my_catalog.warehouse",
"s3://*****/temp-testing-aa")
.config ("spark.sql.catalog.my_catalog.catalog-impl",
"org.apache.iceberg.aws.glue.GlueCatalog")
.config("spark.sql.catalog.my_catalog.io-impl",
"org.apache.iceberg.aws.s3.S3FileIO")
.getOrCreate()
val icebergDDL =
"""
|CREATE TABLE IF NOT EXISTS my_catalog.default.iceberg_demo (
| email string,
| data string,
| category string)
|USING iceberg
|PARTITIONED BY (category);
|""".stripMargin
spark.sql(icebergDDL)
val icebergInsert =
"""
|INSERT INTO my_catalog.default.iceberg_demo (email, data, category)
|VALUES
|('AABB1', 'data 1' , 'CAT1'),
|('AABB2', 'data 2' , 'CAT1'),
|('AABB3', 'data 3' , 'CAT2'),
|('AABB4', 'data 4' , 'CAT3')
|""".stripMargin
spark.sql(icebergInsert)
val df = spark.table("my_catalog.default.iceberg_demo")
df.show()
```
Table has been created and viewable from Athena and Spark.
Use Athena to insert a record
```
INSERT INTO iceberg_demo (email, data, category) VALUES ('AABB5', 'data 5'
, 'CAT5')
```
Run a spark job again with the following line
```
val df = spark.table("my_catalog.default.iceberg_demo")
df.show()
```
The following exception is thrown
```
22/08/20 22:48:26 INFO CatalogUtil: Loading custom FileIO implementation:
org.apache.iceberg.aws.s3.S3FileIO
22/08/20 22:48:27 INFO BaseMetastoreTableOperations: Refreshing table
metadata from new version:
s3://af-biz-dev/temp-testing-aa/default.db/iceberg_demo/metadata/00002-59d72a1f-8b73-4386-bb86-3b6dfc29e858.metadata.json
22/08/20 22:48:28 INFO BaseMetastoreCatalog: Table loaded by catalog:
my_catalog.default.iceberg_demo
22/08/20 22:48:28 INFO V2ScanRelationPushDown:
Output: email#0, data#1, category#2
......
.....
22/08/20 22:48:32 ERROR BaseDataReader: Error reading file:
s3://af-biz-dev/temp-testing-aa/default.db/iceberg_demo/data/category=CAT5/ff78aec0-344e-489a-9206-ab977cd9897b.gz.parquet
java.lang.UnsupportedOperationException: Cannot support vectorized reads for
column [email] optional binary email (STRING) = 1 with encoding
DELTA_BYTE_ARRAY. Disable vectorized reads to read this table/file
at
org.apache.iceberg.arrow.vectorized.parquet.VectorizedPageIterator.initDataReader(VectorizedPageIterator.java:96)
at
org.apache.iceberg.parquet.BasePageIterator.initFromPage(BasePageIterator.java:136)
at
org.apache.iceberg.parquet.BasePageIterator$1.visit(BasePageIterator.java:103)
at
org.apache.iceberg.parquet.BasePageIterator$1.visit(BasePageIterator.java:94)
```
This also happens when I try to use
config("spark.sql.parquet.enableVectorizedReader", "false")
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]