Skip to content

Commit 9386419

Browse files
committed
ALP: Add cross-language interop test reading C++ ALP-encoded parquet files
Reads ALP-encoded parquet files (spotify1: 9 columns x 15000 rows, arade: 4 columns x 15000 rows) generated by the C++ Arrow implementation and verifies all double values match the expected CSV data bit-exactly. This validates the full end-to-end ALP read path through the parquet-java reader stack: file metadata parsing, ALP encoding detection, and AlpValuesReaderForDouble decoding. Note: requires parquet-format Encoding.ALP(10) which is not yet upstream in the parquet-format Thrift spec. Build with -Dmdep.skip=true when rebuilding parquet-format-structures to preserve the local ALP addition.
1 parent f49fa1d commit 9386419

5 files changed

Lines changed: 30191 additions & 0 deletions

File tree

Lines changed: 189 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,189 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
package org.apache.parquet.hadoop;
20+
21+
import static org.junit.Assert.assertEquals;
22+
import static org.junit.Assert.assertNotNull;
23+
24+
import java.io.BufferedReader;
25+
import java.io.IOException;
26+
import java.io.InputStream;
27+
import java.io.InputStreamReader;
28+
import java.net.URISyntaxException;
29+
import java.nio.charset.StandardCharsets;
30+
import java.util.ArrayList;
31+
import java.util.List;
32+
import org.apache.hadoop.fs.Path;
33+
import org.apache.parquet.column.Encoding;
34+
import org.apache.parquet.example.data.Group;
35+
import org.apache.parquet.hadoop.example.GroupReadSupport;
36+
import org.apache.parquet.hadoop.metadata.BlockMetaData;
37+
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
38+
import org.junit.Test;
39+
40+
/**
41+
* Integration test for reading ALP (Adaptive Lossless floating-Point) encoded
42+
* parquet files generated by the C++ implementation and verifying correctness
43+
* against expected CSV data.
44+
*
45+
* <p>The test parquet files were generated using the generate_alp_parquet C++
46+
* utility from Arrow, which encodes floating-point CSV datasets using ALP encoding.
47+
*/
48+
public class TestInteropAlpEncoding {
49+
50+
private static Path resourcePath(String name) {
51+
try {
52+
return new Path(TestInteropAlpEncoding.class.getResource("/" + name).toURI());
53+
} catch (URISyntaxException e) {
54+
throw new RuntimeException(e);
55+
}
56+
}
57+
58+
/**
59+
* Read the ALP-encoded arade parquet file (4 double columns, 15000 rows)
60+
* and verify all values match the expected CSV.
61+
*/
62+
@Test
63+
public void testReadAlpAradeParquet() throws IOException {
64+
Path parquetPath = resourcePath("alp_arade.parquet");
65+
String[] columnNames = {"value1", "value2", "value3", "value4"};
66+
int expectedRows = 15000;
67+
68+
// Read expected values from CSV
69+
double[][] expected = readExpectedCsv("/alp_arade_expect.csv", columnNames.length, expectedRows);
70+
71+
// Read parquet file using GroupReadSupport
72+
List<Group> rows = readParquetGroups(parquetPath);
73+
assertEquals("Row count should match", expectedRows, rows.size());
74+
75+
// Verify ALP encoding is used in metadata
76+
verifyAlpEncoding(parquetPath);
77+
78+
// Compare all values
79+
for (int r = 0; r < expectedRows; r++) {
80+
Group group = rows.get(r);
81+
for (int c = 0; c < columnNames.length; c++) {
82+
double actual = group.getDouble(columnNames[c], 0);
83+
assertEquals(
84+
String.format("Mismatch at row %d, column %s", r, columnNames[c]),
85+
Double.doubleToLongBits(expected[c][r]),
86+
Double.doubleToLongBits(actual));
87+
}
88+
}
89+
}
90+
91+
/**
92+
* Read the ALP-encoded spotify1 parquet file (9 double columns, 15000 rows)
93+
* and verify all values match the expected CSV.
94+
*/
95+
@Test
96+
public void testReadAlpSpotify1Parquet() throws IOException {
97+
Path parquetPath = resourcePath("alp_spotify1.parquet");
98+
String[] columnNames = {
99+
"danceability",
100+
"energy",
101+
"loudness",
102+
"speechiness",
103+
"acousticness",
104+
"instrumentalness",
105+
"liveness",
106+
"valence",
107+
"tempo"
108+
};
109+
int expectedRows = 15000;
110+
111+
// Read expected values from CSV
112+
double[][] expected = readExpectedCsv("/alp_spotify1_expect.csv", columnNames.length, expectedRows);
113+
114+
// Read parquet file using GroupReadSupport
115+
List<Group> rows = readParquetGroups(parquetPath);
116+
assertEquals("Row count should match", expectedRows, rows.size());
117+
118+
// Verify ALP encoding is used in metadata
119+
verifyAlpEncoding(parquetPath);
120+
121+
// Compare all values
122+
for (int r = 0; r < expectedRows; r++) {
123+
Group group = rows.get(r);
124+
for (int c = 0; c < columnNames.length; c++) {
125+
double actual = group.getDouble(columnNames[c], 0);
126+
assertEquals(
127+
String.format("Mismatch at row %d, column %s", r, columnNames[c]),
128+
Double.doubleToLongBits(expected[c][r]),
129+
Double.doubleToLongBits(actual));
130+
}
131+
}
132+
}
133+
134+
private List<Group> readParquetGroups(Path path) throws IOException {
135+
List<Group> rows = new ArrayList<>();
136+
try (ParquetReader<Group> reader =
137+
ParquetReader.builder(new GroupReadSupport(), path).build()) {
138+
Group group;
139+
while ((group = reader.read()) != null) {
140+
rows.add(group);
141+
}
142+
}
143+
return rows;
144+
}
145+
146+
private void verifyAlpEncoding(Path path) throws IOException {
147+
try (ParquetFileReader reader = ParquetFileReader.open(org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(
148+
path, new org.apache.hadoop.conf.Configuration()))) {
149+
List<BlockMetaData> blocks = reader.getFooter().getBlocks();
150+
for (BlockMetaData block : blocks) {
151+
for (ColumnChunkMetaData column : block.getColumns()) {
152+
assertNotNull(
153+
"Column " + column.getPath() + " should have encoding stats", column.getEncodingStats());
154+
boolean hasAlp = column.getEncodings().contains(Encoding.ALP);
155+
assertEquals("Column " + column.getPath() + " should use ALP encoding", true, hasAlp);
156+
}
157+
}
158+
}
159+
}
160+
161+
/**
162+
* Parse expected CSV into column arrays.
163+
* CSV format: header row, then data rows with comma-separated double values.
164+
*/
165+
private double[][] readExpectedCsv(String resourcePath, int numColumns, int expectedRows) throws IOException {
166+
double[][] columns = new double[numColumns][expectedRows];
167+
try (InputStream is = getClass().getResourceAsStream(resourcePath);
168+
BufferedReader br = new BufferedReader(new InputStreamReader(is, StandardCharsets.UTF_8))) {
169+
assertNotNull("CSV resource not found: " + resourcePath, is);
170+
171+
// Skip header
172+
String header = br.readLine();
173+
assertNotNull("CSV should have a header", header);
174+
175+
int row = 0;
176+
String line;
177+
while ((line = br.readLine()) != null) {
178+
String[] parts = line.split(",");
179+
assertEquals("CSV row " + row + " should have " + numColumns + " columns", numColumns, parts.length);
180+
for (int c = 0; c < numColumns; c++) {
181+
columns[c][row] = Double.parseDouble(parts[c]);
182+
}
183+
row++;
184+
}
185+
assertEquals("CSV should have " + expectedRows + " data rows", expectedRows, row);
186+
}
187+
return columns;
188+
}
189+
}
176 KB
Binary file not shown.

0 commit comments

Comments
 (0)