Skip to content

Commit f49fa1d

Browse files
committed
ALP: Apply spotless formatting fixes
1 parent 5c011f6 commit f49fa1d

11 files changed

Lines changed: 213 additions & 293 deletions

File tree

parquet-column/src/main/java/org/apache/parquet/column/values/alp/AlpCompression.java

Lines changed: 4 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -89,8 +89,7 @@ void store(byte[] output, int offset) {
8989
}
9090

9191
void storeDataOnly(byte[] output, int offset) {
92-
ByteBuffer buf =
93-
ByteBuffer.wrap(output, offset, dataStoredSize()).order(ByteOrder.LITTLE_ENDIAN);
92+
ByteBuffer buf = ByteBuffer.wrap(output, offset, dataStoredSize()).order(ByteOrder.LITTLE_ENDIAN);
9493
storeDataTo(buf);
9594
}
9695

@@ -169,8 +168,7 @@ void store(byte[] output, int offset) {
169168
}
170169

171170
void storeDataOnly(byte[] output, int offset) {
172-
ByteBuffer buf =
173-
ByteBuffer.wrap(output, offset, dataStoredSize()).order(ByteOrder.LITTLE_ENDIAN);
171+
ByteBuffer buf = ByteBuffer.wrap(output, offset, dataStoredSize()).order(ByteOrder.LITTLE_ENDIAN);
174172
storeDataTo(buf);
175173
}
176174

@@ -212,8 +210,7 @@ static DoubleCompressedVector load(byte[] input, int offset, int numElements) {
212210

213211
// ========== Compress float ==========
214212

215-
static FloatCompressedVector compressFloatVector(
216-
float[] input, int count, AlpEncodingPreset preset) {
213+
static FloatCompressedVector compressFloatVector(float[] input, int count, AlpEncodingPreset preset) {
217214
if (count == 0) {
218215
FloatCompressedVector r = new FloatCompressedVector();
219216
r.packedValues = new byte[0];
@@ -321,8 +318,7 @@ static void decompressFloatVector(FloatCompressedVector v, float[] output) {
321318

322319
// ========== Compress double ==========
323320

324-
static DoubleCompressedVector compressDoubleVector(
325-
double[] input, int count, AlpEncodingPreset preset) {
321+
static DoubleCompressedVector compressDoubleVector(double[] input, int count, AlpEncodingPreset preset) {
326322
if (count == 0) {
327323
DoubleCompressedVector r = new DoubleCompressedVector();
328324
r.packedValues = new byte[0];

parquet-column/src/main/java/org/apache/parquet/column/values/alp/AlpConstants.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -88,8 +88,8 @@ private AlpConstants() {
8888

8989
// Negative powers of 10 as double, matching C++ PowerOfTenDouble(-power).
9090
static final double[] DOUBLE_POW10_NEGATIVE = {
91-
1e0, 1e-1, 1e-2, 1e-3, 1e-4, 1e-5, 1e-6, 1e-7, 1e-8, 1e-9, 1e-10,
92-
1e-11, 1e-12, 1e-13, 1e-14, 1e-15, 1e-16, 1e-17, 1e-18
91+
1e0, 1e-1, 1e-2, 1e-3, 1e-4, 1e-5, 1e-6, 1e-7, 1e-8, 1e-9, 1e-10, 1e-11, 1e-12, 1e-13, 1e-14, 1e-15, 1e-16,
92+
1e-17, 1e-18
9393
};
9494

9595
// ========== Per-vector metadata sizes ==========

parquet-column/src/main/java/org/apache/parquet/column/values/alp/AlpSampler.java

Lines changed: 4 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,6 @@
2121
import static org.apache.parquet.column.values.alp.AlpConstants.*;
2222

2323
import java.util.ArrayList;
24-
import java.util.Arrays;
2524
import java.util.HashMap;
2625
import java.util.List;
2726
import java.util.Map;
@@ -63,8 +62,7 @@ void addSample(float[] data, int count) {
6362
}
6463

6564
private void addSampleVector(float[] data, int offset, int length) {
66-
boolean mustSkip = mustSkipSamplingFromCurrentVector(
67-
vectorsCount, vectorsSampledCount, length);
65+
boolean mustSkip = mustSkipSamplingFromCurrentVector(vectorsCount, vectorsSampledCount, length);
6866
vectorsCount++;
6967
totalValuesCount += length;
7068
if (mustSkip) {
@@ -129,8 +127,7 @@ void addSample(double[] data, int count) {
129127
}
130128

131129
private void addSampleVector(double[] data, int offset, int length) {
132-
boolean mustSkip = mustSkipSamplingFromCurrentVector(
133-
vectorsCount, vectorsSampledCount, length);
130+
boolean mustSkip = mustSkipSamplingFromCurrentVector(vectorsCount, vectorsSampledCount, length);
134131
vectorsCount++;
135132
totalValuesCount += length;
136133
if (mustSkip) {
@@ -209,8 +206,7 @@ private static long estimateFloatCompressedSize(
209206
return estimatedSize;
210207
}
211208

212-
static AlpCompression.AlpEncodingPreset createFloatEncodingPreset(
213-
List<float[]> vectorsSampled) {
209+
static AlpCompression.AlpEncodingPreset createFloatEncodingPreset(List<float[]> vectorsSampled) {
214210
// For each sampled vector, find the best (e,f) combo by estimated compressed size.
215211
// Count how many times each best combo appears across all sampled vectors.
216212
Map<Long, int[]> bestCombosCount = new HashMap<>(); // key = e<<8|f, value = [count]
@@ -299,8 +295,7 @@ private static long estimateDoubleCompressedSize(
299295
return estimatedSize;
300296
}
301297

302-
static AlpCompression.AlpEncodingPreset createDoubleEncodingPreset(
303-
List<double[]> vectorsSampled) {
298+
static AlpCompression.AlpEncodingPreset createDoubleEncodingPreset(List<double[]> vectorsSampled) {
304299
Map<Long, int[]> bestCombosCount = new HashMap<>();
305300

306301
for (double[] sample : vectorsSampled) {

parquet-column/src/main/java/org/apache/parquet/column/values/alp/AlpValuesReader.java

Lines changed: 4 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -48,8 +48,7 @@ abstract class AlpValuesReader extends ValuesReader {
4848
public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IOException {
4949
int available = (int) stream.available();
5050
if (available < HEADER_SIZE) {
51-
throw new ParquetDecodingException(
52-
"ALP page too small for header: " + available + " bytes");
51+
throw new ParquetDecodingException("ALP page too small for header: " + available + " bytes");
5352
}
5453

5554
// Read header
@@ -63,12 +62,10 @@ public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IO
6362
totalCount = header.getInt();
6463

6564
if (compressionMode != COMPRESSION_MODE_ALP) {
66-
throw new ParquetDecodingException(
67-
"Unsupported ALP compression mode: " + compressionMode);
65+
throw new ParquetDecodingException("Unsupported ALP compression mode: " + compressionMode);
6866
}
6967
if (integerEncoding != INTEGER_ENCODING_FOR) {
70-
throw new ParquetDecodingException(
71-
"Unsupported ALP integer encoding: " + integerEncoding);
68+
throw new ParquetDecodingException("Unsupported ALP integer encoding: " + integerEncoding);
7269
}
7370

7471
vectorSize = 1 << logVectorSize;
@@ -88,8 +85,7 @@ public void initFromPage(int valueCount, ByteBufferInputStream stream) throws IO
8885
stream.read(rawData);
8986

9087
// Parse offsets from rawData
91-
ByteBuffer body = ByteBuffer.wrap(rawData, 0, numVectors * OFFSET_SIZE)
92-
.order(ByteOrder.LITTLE_ENDIAN);
88+
ByteBuffer body = ByteBuffer.wrap(rawData, 0, numVectors * OFFSET_SIZE).order(ByteOrder.LITTLE_ENDIAN);
9389
vectorOffsets = new int[numVectors];
9490
for (int i = 0; i < numVectors; i++) {
9591
vectorOffsets[i] = body.getInt();

parquet-column/src/main/java/org/apache/parquet/column/values/alp/AlpWrapper.java

Lines changed: 32 additions & 45 deletions
Original file line numberDiff line numberDiff line change
@@ -47,16 +47,14 @@ private AlpWrapper() {}
4747
// ========== Sampling presets ==========
4848

4949
/** Create a sampling-based encoding preset for float data. */
50-
public static AlpCompression.AlpEncodingPreset createFloatSamplingPreset(
51-
float[] data, int count) {
50+
public static AlpCompression.AlpEncodingPreset createFloatSamplingPreset(float[] data, int count) {
5251
AlpSampler.FloatSampler sampler = new AlpSampler.FloatSampler();
5352
sampler.addSample(data, count);
5453
return sampler.finalizeSampling();
5554
}
5655

5756
/** Create a sampling-based encoding preset for double data. */
58-
public static AlpCompression.AlpEncodingPreset createDoubleSamplingPreset(
59-
double[] data, int count) {
57+
public static AlpCompression.AlpEncodingPreset createDoubleSamplingPreset(double[] data, int count) {
6058
AlpSampler.DoubleSampler sampler = new AlpSampler.DoubleSampler();
6159
sampler.addSample(data, count);
6260
return sampler.finalizeSampling();
@@ -73,12 +71,10 @@ public static AlpCompression.AlpEncodingPreset createDoubleSamplingPreset(
7371
* @param preset the encoding preset from sampling
7472
* @return the number of compressed bytes written
7573
*/
76-
public static int encodeFloats(
77-
float[] input, int count, byte[] output, AlpCompression.AlpEncodingPreset preset) {
74+
public static int encodeFloats(float[] input, int count, byte[] output, AlpCompression.AlpEncodingPreset preset) {
7875
Preconditions.checkArgument(count >= 0, "count must be non-negative, got: %s", count);
7976
if (count == 0) {
80-
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR,
81-
DEFAULT_VECTOR_SIZE_LOG, 0);
77+
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR, DEFAULT_VECTOR_SIZE_LOG, 0);
8278
return HEADER_SIZE;
8379
}
8480

@@ -101,14 +97,14 @@ public static int encodeFloats(
10197
int currentOffset = offsetsSectionSize;
10298
for (int i = 0; i < numVectors; i++) {
10399
vectorOffsets[i] = currentOffset;
104-
currentOffset += ALP_INFO_SIZE + FLOAT_FOR_INFO_SIZE + vectors.get(i).dataStoredSize();
100+
currentOffset +=
101+
ALP_INFO_SIZE + FLOAT_FOR_INFO_SIZE + vectors.get(i).dataStoredSize();
105102
}
106103
int bodySize = currentOffset;
107104
int totalSize = HEADER_SIZE + bodySize;
108105

109106
// Phase 3: Write header
110-
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR,
111-
DEFAULT_VECTOR_SIZE_LOG, count);
107+
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR, DEFAULT_VECTOR_SIZE_LOG, count);
112108

113109
// Phase 4: Write offsets
114110
ByteBuffer buf = ByteBuffer.wrap(output, HEADER_SIZE, bodySize).order(ByteOrder.LITTLE_ENDIAN);
@@ -128,12 +124,10 @@ public static int encodeFloats(
128124

129125
// ========== Encode doubles ==========
130126

131-
public static int encodeDoubles(
132-
double[] input, int count, byte[] output, AlpCompression.AlpEncodingPreset preset) {
127+
public static int encodeDoubles(double[] input, int count, byte[] output, AlpCompression.AlpEncodingPreset preset) {
133128
Preconditions.checkArgument(count >= 0, "count must be non-negative, got: %s", count);
134129
if (count == 0) {
135-
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR,
136-
DEFAULT_VECTOR_SIZE_LOG, 0);
130+
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR, DEFAULT_VECTOR_SIZE_LOG, 0);
137131
return HEADER_SIZE;
138132
}
139133

@@ -146,8 +140,7 @@ public static int encodeDoubles(
146140
int elementsInVector = Math.min(vectorSize, count - offset);
147141
double[] vectorInput = new double[elementsInVector];
148142
System.arraycopy(input, offset, vectorInput, 0, elementsInVector);
149-
vectors.add(
150-
AlpCompression.compressDoubleVector(vectorInput, elementsInVector, preset));
143+
vectors.add(AlpCompression.compressDoubleVector(vectorInput, elementsInVector, preset));
151144
}
152145

153146
int offsetsSectionSize = numVectors * OFFSET_SIZE;
@@ -161,8 +154,7 @@ public static int encodeDoubles(
161154
int bodySize = currentOffset;
162155
int totalSize = HEADER_SIZE + bodySize;
163156

164-
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR,
165-
DEFAULT_VECTOR_SIZE_LOG, count);
157+
writeHeader(output, 0, COMPRESSION_MODE_ALP, INTEGER_ENCODING_FOR, DEFAULT_VECTOR_SIZE_LOG, count);
166158

167159
ByteBuffer buf = ByteBuffer.wrap(output, HEADER_SIZE, bodySize).order(ByteOrder.LITTLE_ENDIAN);
168160
for (int offset : vectorOffsets) {
@@ -188,30 +180,28 @@ public static int encodeDoubles(
188180
* @param output output float array (must hold numElements values)
189181
* @param numElements number of elements to decode
190182
*/
191-
public static void decodeFloats(
192-
byte[] compressed, int compSize, float[] output, int numElements) {
193-
Preconditions.checkArgument(compSize >= HEADER_SIZE,
194-
"compressed size too small for header: %s", compSize);
183+
public static void decodeFloats(byte[] compressed, int compSize, float[] output, int numElements) {
184+
Preconditions.checkArgument(compSize >= HEADER_SIZE, "compressed size too small for header: %s", compSize);
195185

196186
ByteBuffer header = ByteBuffer.wrap(compressed, 0, HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN);
197187
int compressionMode = header.get() & 0xFF;
198188
int integerEncoding = header.get() & 0xFF;
199189
int logVectorSize = header.get() & 0xFF;
200190
int storedNumElements = header.getInt();
201191

202-
Preconditions.checkArgument(compressionMode == COMPRESSION_MODE_ALP,
203-
"unsupported compression mode: %s", compressionMode);
204-
Preconditions.checkArgument(integerEncoding == INTEGER_ENCODING_FOR,
205-
"unsupported integer encoding: %s", integerEncoding);
192+
Preconditions.checkArgument(
193+
compressionMode == COMPRESSION_MODE_ALP, "unsupported compression mode: %s", compressionMode);
194+
Preconditions.checkArgument(
195+
integerEncoding == INTEGER_ENCODING_FOR, "unsupported integer encoding: %s", integerEncoding);
206196

207197
int vectorSize = 1 << logVectorSize;
208198
int numVectors = (storedNumElements + vectorSize - 1) / vectorSize;
209199

210200
if (numVectors == 0) return;
211201

212202
// Read offsets
213-
ByteBuffer body = ByteBuffer.wrap(compressed, HEADER_SIZE, compSize - HEADER_SIZE)
214-
.order(ByteOrder.LITTLE_ENDIAN);
203+
ByteBuffer body =
204+
ByteBuffer.wrap(compressed, HEADER_SIZE, compSize - HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN);
215205
int[] vectorOffsets = new int[numVectors];
216206
for (int i = 0; i < numVectors; i++) {
217207
vectorOffsets[i] = body.getInt();
@@ -234,37 +224,35 @@ public static void decodeFloats(
234224

235225
float[] vectorOutput = new float[elementsInVector];
236226
AlpCompression.decompressFloatVector(cv, vectorOutput);
237-
System.arraycopy(vectorOutput, 0, output, outputOffset,
238-
Math.min(elementsInVector, numElements - outputOffset));
227+
System.arraycopy(
228+
vectorOutput, 0, output, outputOffset, Math.min(elementsInVector, numElements - outputOffset));
239229
outputOffset += elementsInVector;
240230
}
241231
}
242232

243233
// ========== Decode doubles ==========
244234

245-
public static void decodeDoubles(
246-
byte[] compressed, int compSize, double[] output, int numElements) {
247-
Preconditions.checkArgument(compSize >= HEADER_SIZE,
248-
"compressed size too small for header: %s", compSize);
235+
public static void decodeDoubles(byte[] compressed, int compSize, double[] output, int numElements) {
236+
Preconditions.checkArgument(compSize >= HEADER_SIZE, "compressed size too small for header: %s", compSize);
249237

250238
ByteBuffer header = ByteBuffer.wrap(compressed, 0, HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN);
251239
int compressionMode = header.get() & 0xFF;
252240
int integerEncoding = header.get() & 0xFF;
253241
int logVectorSize = header.get() & 0xFF;
254242
int storedNumElements = header.getInt();
255243

256-
Preconditions.checkArgument(compressionMode == COMPRESSION_MODE_ALP,
257-
"unsupported compression mode: %s", compressionMode);
258-
Preconditions.checkArgument(integerEncoding == INTEGER_ENCODING_FOR,
259-
"unsupported integer encoding: %s", integerEncoding);
244+
Preconditions.checkArgument(
245+
compressionMode == COMPRESSION_MODE_ALP, "unsupported compression mode: %s", compressionMode);
246+
Preconditions.checkArgument(
247+
integerEncoding == INTEGER_ENCODING_FOR, "unsupported integer encoding: %s", integerEncoding);
260248

261249
int vectorSize = 1 << logVectorSize;
262250
int numVectors = (storedNumElements + vectorSize - 1) / vectorSize;
263251

264252
if (numVectors == 0) return;
265253

266-
ByteBuffer body = ByteBuffer.wrap(compressed, HEADER_SIZE, compSize - HEADER_SIZE)
267-
.order(ByteOrder.LITTLE_ENDIAN);
254+
ByteBuffer body =
255+
ByteBuffer.wrap(compressed, HEADER_SIZE, compSize - HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN);
268256
int[] vectorOffsets = new int[numVectors];
269257
for (int i = 0; i < numVectors; i++) {
270258
vectorOffsets[i] = body.getInt();
@@ -286,8 +274,8 @@ public static void decodeDoubles(
286274

287275
double[] vectorOutput = new double[elementsInVector];
288276
AlpCompression.decompressDoubleVector(cv, vectorOutput);
289-
System.arraycopy(vectorOutput, 0, output, outputOffset,
290-
Math.min(elementsInVector, numElements - outputOffset));
277+
System.arraycopy(
278+
vectorOutput, 0, output, outputOffset, Math.min(elementsInVector, numElements - outputOffset));
291279
outputOffset += elementsInVector;
292280
}
293281
}
@@ -322,8 +310,7 @@ public static long maxCompressedSizeDouble(int numElements) {
322310
// ========== Header helpers ==========
323311

324312
private static void writeHeader(
325-
byte[] output, int offset, int compressionMode, int integerEncoding,
326-
int logVectorSize, int numElements) {
313+
byte[] output, int offset, int compressionMode, int integerEncoding, int logVectorSize, int numElements) {
327314
ByteBuffer buf = ByteBuffer.wrap(output, offset, HEADER_SIZE).order(ByteOrder.LITTLE_ENDIAN);
328315
buf.put((byte) compressionMode);
329316
buf.put((byte) integerEncoding);

0 commit comments

Comments
 (0)