Skip to content

Commit a0983cc

Browse files
committed
update
1 parent 36a5f9c commit a0983cc

5 files changed

Lines changed: 145 additions & 2 deletions

File tree

parquet-cli/README.md

Lines changed: 22 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -134,3 +134,25 @@ Usage: parquet [options] [command] [command options]
134134
See 'parquet help <command>' for more information on a specific command.
135135
```
136136

137+
### Configuration Options
138+
139+
The `convert` and `convert-csv` commands support a generic configuration mechanism:
140+
141+
- `--conf` or `--property`: Set any configuration property in format `key=value`. Can be specified multiple times.
142+
143+
This allows you to configure any Avro or Parquet setting without needing to modify source code or rebuild the project.
144+
145+
Examples:
146+
```bash
147+
# Enable UUID support
148+
parquet convert input.avro -o output.parquet --conf parquet.avro.write-parquet-uuid=true
149+
150+
# Use new 3-level list structure
151+
parquet convert input.avro -o output.parquet --conf parquet.avro.write-old-list-structure=false
152+
153+
# Convert CSV with multiple options
154+
parquet convert-csv input.csv -o output.parquet --schema schema.avsc --conf parquet.avro.write-parquet-uuid=true --conf parquet.avro.write-old-list-structure=false
155+
156+
# Set any other configuration properties
157+
parquet convert input.avro -o output.parquet --conf parquet.avro.add-list-element-records=false --conf parquet.avro.write.data.supplier=org.apache.parquet.avro.GenericDataSupplier
158+
```

parquet-cli/src/main/java/org/apache/parquet/cli/commands/ConvertCSVCommand.java

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -45,6 +45,7 @@
4545
import org.apache.parquet.cli.util.Schemas;
4646
import org.apache.parquet.hadoop.ParquetFileWriter;
4747
import org.apache.parquet.hadoop.ParquetWriter;
48+
import org.apache.hadoop.conf.Configuration;
4849
import org.slf4j.Logger;
4950

5051
@Parameters(commandDescription = "Create a file from CSV data")
@@ -117,6 +118,11 @@ public ConvertCSVCommand(Logger console) {
117118
description = "Remove any data already in the target view or dataset")
118119
boolean overwrite = false;
119120

121+
@Parameter(
122+
names = {"--conf", "--property"},
123+
description = "Set a configuration property (format: key=value). Can be specified multiple times.")
124+
List<String> confProperties;
125+
120126
@Override
121127
@SuppressWarnings("unchecked")
122128
public int run() throws IOException {
@@ -168,6 +174,23 @@ public int run() throws IOException {
168174
}
169175
}
170176

177+
// Create a configuration and apply custom properties
178+
Configuration conf = new Configuration(getConf());
179+
180+
// Apply custom configuration properties
181+
if (confProperties != null) {
182+
for (String prop : confProperties) {
183+
String[] parts = prop.split("=", 2);
184+
if (parts.length != 2) {
185+
throw new IllegalArgumentException("Configuration property must be in format key=value: " + prop);
186+
}
187+
String key = parts[0].trim();
188+
String value = parts[1].trim();
189+
conf.set(key, value);
190+
console.debug("Set configuration property: {}={}", key, value);
191+
}
192+
}
193+
171194
try (ParquetWriter<Record> writer = AvroParquetWriter.<Record>builder(qualifiedPath(outputPath))
172195
.withWriterVersion(v2 ? PARQUET_2_0 : PARQUET_1_0)
173196
.withWriteMode(overwrite ? ParquetFileWriter.Mode.OVERWRITE : ParquetFileWriter.Mode.CREATE)
@@ -177,7 +200,7 @@ public int run() throws IOException {
177200
.withPageSize(pageSize)
178201
.withRowGroupSize(rowGroupSize)
179202
.withDataModel(GenericData.get())
180-
.withConf(getConf())
203+
.withConf(conf)
181204
.withSchema(csvSchema)
182205
.build()) {
183206
for (String target : targets) {

parquet-cli/src/main/java/org/apache/parquet/cli/commands/ConvertCommand.java

Lines changed: 24 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@
4343
import org.apache.parquet.hadoop.ParquetWriter;
4444
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
4545
import org.slf4j.Logger;
46+
import org.apache.hadoop.conf.Configuration;
4647

4748
@Parameters(commandDescription = "Create a Parquet file from a data file")
4849
public class ConvertCommand extends BaseCommand {
@@ -95,6 +96,11 @@ public ConvertCommand(Logger console) {
9596
@Parameter(names = "--dictionary-size", description = "Max dictionary page size")
9697
int dictionaryPageSize = ParquetWriter.DEFAULT_PAGE_SIZE;
9798

99+
@Parameter(
100+
names = {"--conf", "--property"},
101+
description = "Set a configuration property (format: key=value). Can be specified multiple times.")
102+
List<String> confProperties;
103+
98104
@Override
99105
@SuppressWarnings("unchecked")
100106
public int run() throws IOException {
@@ -119,13 +125,30 @@ public int run() throws IOException {
119125
outFS.delete(outPath);
120126
}
121127

128+
// Create a configuration and apply custom properties
129+
Configuration conf = new Configuration(getConf());
130+
131+
// Apply custom configuration properties
132+
if (confProperties != null) {
133+
for (String prop : confProperties) {
134+
String[] parts = prop.split("=", 2);
135+
if (parts.length != 2) {
136+
throw new IllegalArgumentException("Configuration property must be in format key=value: " + prop);
137+
}
138+
String key = parts[0].trim();
139+
String value = parts[1].trim();
140+
conf.set(key, value);
141+
console.debug("Set configuration property: {}={}", key, value);
142+
}
143+
}
144+
122145
Iterable<Record> reader = openDataFile(source, projection);
123146
boolean threw = true;
124147
long count = 0;
125148
try {
126149
try (ParquetWriter<Record> writer = AvroParquetWriter.<Record>builder(qualifiedPath(outputPath))
127150
.withWriterVersion(v2 ? PARQUET_2_0 : PARQUET_1_0)
128-
.withConf(getConf())
151+
.withConf(conf)
129152
.withCompressionCodec(codec)
130153
.withRowGroupSize(rowGroupSize)
131154
.withDictionaryPageSize(dictionaryPageSize < 64 ? 64 : dictionaryPageSize)

parquet-cli/src/test/java/org/apache/parquet/cli/commands/ConvertCSVCommandTest.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -61,4 +61,20 @@ public void testConvertCSVCommandWithDifferentSchemas() throws IOException {
6161
command.setConf(new Configuration());
6262
command.run();
6363
}
64+
65+
@Test
66+
public void testConvertCSVCommandWithGenericConf() throws IOException {
67+
File file = csvFile();
68+
ConvertCSVCommand command = new ConvertCSVCommand(createLogger());
69+
command.targets = Arrays.asList(file.getAbsolutePath());
70+
File output = new File(getTempFolder(), getClass().getSimpleName() + "_with_generic_conf.parquet");
71+
command.outputPath = output.getAbsolutePath();
72+
command.confProperties = Arrays.asList(
73+
"parquet.avro.write-parquet-uuid=true",
74+
"parquet.avro.write-old-list-structure=false"
75+
);
76+
command.setConf(new Configuration());
77+
Assert.assertEquals(0, command.run());
78+
Assert.assertTrue(output.exists());
79+
}
6480
}

parquet-cli/src/test/java/org/apache/parquet/cli/commands/ConvertCommandTest.java

Lines changed: 59 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -37,4 +37,63 @@ public void testConvertCommand() throws IOException {
3737
Assert.assertEquals(0, command.run());
3838
Assert.assertTrue(output.exists());
3939
}
40+
41+
@Test
42+
public void testConvertCommandWithGenericConf() throws IOException {
43+
File file = toAvro(parquetFile());
44+
ConvertCommand command = new ConvertCommand(createLogger());
45+
command.targets = Arrays.asList(file.getAbsolutePath());
46+
File output = new File(getTempFolder(), "converted_with_generic_conf.parquet");
47+
command.outputPath = output.getAbsolutePath();
48+
command.confProperties = Arrays.asList(
49+
"parquet.avro.write-parquet-uuid=true",
50+
"parquet.avro.write-old-list-structure=false",
51+
"test.property=test.value"
52+
);
53+
command.setConf(new Configuration());
54+
55+
Assert.assertEquals(0, command.run());
56+
Assert.assertTrue(output.exists());
57+
}
58+
59+
@Test
60+
public void testConvertCommandConfigurationValidation() throws IOException {
61+
File file = toAvro(parquetFile());
62+
ConvertCommand command = new ConvertCommand(createLogger());
63+
command.targets = Arrays.asList(file.getAbsolutePath());
64+
File output = new File(getTempFolder(), "converted_with_config_validation.parquet");
65+
command.outputPath = output.getAbsolutePath();
66+
67+
command.confProperties = Arrays.asList(
68+
"parquet.avro.write-parquet-uuid=true",
69+
"parquet.avro.write-old-list-structure=false"
70+
);
71+
72+
command.setConf(new Configuration());
73+
74+
Assert.assertEquals(0, command.run());
75+
Assert.assertTrue(output.exists());
76+
77+
File output2 = new File(getTempFolder(), "converted_with_config_validation2.parquet");
78+
command.outputPath = output2.getAbsolutePath();
79+
command.confProperties = Arrays.asList(
80+
"parquet.avro.write-parquet-uuid=false",
81+
"parquet.avro.write-old-list-structure=true"
82+
);
83+
84+
Assert.assertEquals(0, command.run());
85+
Assert.assertTrue(output2.exists());
86+
}
87+
88+
@Test(expected = IllegalArgumentException.class)
89+
public void testConvertCommandWithInvalidConf() throws IOException {
90+
File file = toAvro(parquetFile());
91+
ConvertCommand command = new ConvertCommand(createLogger());
92+
command.targets = Arrays.asList(file.getAbsolutePath());
93+
File output = new File(getTempFolder(), "converted_with_invalid_conf.parquet");
94+
command.outputPath = output.getAbsolutePath();
95+
command.confProperties = Arrays.asList("invalid-property-format");
96+
command.setConf(new Configuration());
97+
command.run();
98+
}
4099
}

0 commit comments

Comments
 (0)