├── .gitignore ├── README.md ├── parquet-compat-1.0.0 ├── .gitignore ├── pom.xml └── src ├── parquet-compat-1.1.0 ├── pom.xml └── src ├── parquet-compat-1.2.0 ├── pom.xml └── src ├── parquet-compat └── src │ └── test │ └── java │ └── parquet │ └── compat │ └── test │ ├── ConvertUtils.java │ ├── CsvParquetWriter.java │ ├── CsvWriteSupport.java │ ├── TestBackwardsCompatibility.java │ ├── TestImpalaCompatibility.java │ ├── TestPerfRegression.java │ ├── TestUtils.java │ └── Utils.java ├── parquet-testdata ├── impala │ ├── 1.1.1-GZIP │ │ ├── customer.impala.parquet │ │ └── nation.impala.parquet │ ├── 1.1.1-NONE │ │ ├── customer.impala.parquet │ │ └── nation.impala.parquet │ └── 1.1.1-SNAPPY │ │ ├── customer.impala.parquet │ │ └── nation.impala.parquet └── tpch │ ├── customer.csv │ ├── customer.schema │ ├── customer.sql │ ├── nation.csv │ ├── nation.schema │ └── nation.sql └── pom.xml /.gitignore: -------------------------------------------------------------------------------- 1 | parquet-mr-tests/.classpath 2 | parquet-mr-tests/.project 3 | parquet-mr-tests/.settings/ 4 | parquet-mr-tests/target/ 5 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | parquet-compatibility 2 | ===================== 3 | 4 | compatibility tests to make sur C and Java implementations can read each other 5 | 6 | Submodules: 7 | ----------- 8 | * parquet-compat 9 | * parquet-testdata 10 | * parquet-compat-$version 11 | 12 | parquet-compat 13 | -------------- 14 | This stores the test sources. This is not a maven submodule. 15 | 16 | parquet-testdata 17 | ---------------- 18 | stores all the csvs and impala files (todo discuss impala compatibility test) 19 | 20 | parquet-compat-$version: 21 | ------------------------ 22 | pom.xml has dependency on the corresponding version of parquet-mr. 23 | src is a symlink to sources in parquet-compat (../parquet-compat/src). 24 | -------------------------------------------------------------------------------- /parquet-compat-1.0.0/.gitignore: -------------------------------------------------------------------------------- 1 | *~ 2 | .* 3 | !.gitignore 4 | target 5 | -------------------------------------------------------------------------------- /parquet-compat-1.0.0/pom.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | 5 | parquet 6 | parquet-compat-1.0.0 7 | 1.0.0 8 | jar 9 | 10 | Parquet Compatibility 1.0.0 11 | https://github.com/Parquet/parquet-compatibility 12 | 13 | 14 | UTF-8 15 | 16 | 17 | 18 | 19 | 20 | org.apache.maven.plugins 21 | maven-surefire-plugin 22 | 23 | -Xmx1024m 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | com.twitter 32 | parquet-column 33 | 1.0.0 34 | test 35 | 36 | 37 | 38 | com.twitter 39 | parquet-hadoop 40 | 1.0.0 41 | test 42 | 43 | 44 | 45 | junit 46 | junit 47 | 4.10 48 | test 49 | 50 | 51 | 52 | -------------------------------------------------------------------------------- /parquet-compat-1.0.0/src: -------------------------------------------------------------------------------- 1 | ../parquet-compat/src -------------------------------------------------------------------------------- /parquet-compat-1.1.0/pom.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | 5 | parquet 6 | parquet-compat-1.1.0 7 | 1.1.0 8 | jar 9 | 10 | Parquet Compatibility 1.1.0 11 | https://github.com/Parquet/parquet-compatibility 12 | 13 | 14 | UTF-8 15 | 16 | 17 | 18 | 19 | 20 | org.apache.maven.plugins 21 | maven-surefire-plugin 22 | 23 | -Xmx1024m 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | com.twitter 32 | parquet-column 33 | 1.1.0 34 | test 35 | 36 | 37 | 38 | com.twitter 39 | parquet-hadoop 40 | 1.1.0 41 | test 42 | 43 | 44 | 45 | junit 46 | junit 47 | 4.10 48 | test 49 | 50 | 51 | 52 | -------------------------------------------------------------------------------- /parquet-compat-1.1.0/src: -------------------------------------------------------------------------------- 1 | ../parquet-compat/src -------------------------------------------------------------------------------- /parquet-compat-1.2.0/pom.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | 5 | parquet 6 | parquet-compat-1.2.0 7 | 1.2.0 8 | jar 9 | 10 | Parquet Compatibility 1.2.0 11 | https://github.com/Parquet/parquet-compatibility 12 | 13 | 14 | UTF-8 15 | 16 | 17 | 18 | 19 | 20 | org.apache.maven.plugins 21 | maven-surefire-plugin 22 | 23 | -Xmx1024m 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | com.twitter 32 | parquet-column 33 | 1.1.0 34 | test 35 | 36 | 37 | 38 | com.twitter 39 | parquet-hadoop 40 | 1.1.0 41 | test 42 | 43 | 44 | 45 | junit 46 | junit 47 | 4.10 48 | test 49 | 50 | 51 | 52 | -------------------------------------------------------------------------------- /parquet-compat-1.2.0/src: -------------------------------------------------------------------------------- 1 | ../parquet-compat/src -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/ConvertUtils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import java.io.BufferedReader; 19 | import java.io.BufferedWriter; 20 | import java.io.File; 21 | import java.io.FileReader; 22 | import java.io.FileWriter; 23 | import java.io.IOException; 24 | import java.util.Arrays; 25 | import java.util.regex.Pattern; 26 | 27 | import org.apache.hadoop.conf.Configuration; 28 | import org.apache.hadoop.fs.Path; 29 | 30 | import parquet.Log; 31 | import parquet.Preconditions; 32 | import parquet.column.page.PageReadStore; 33 | import parquet.example.data.Group; 34 | import parquet.example.data.simple.convert.GroupRecordConverter; 35 | import parquet.hadoop.ParquetFileReader; 36 | import parquet.hadoop.ParquetReader; 37 | import parquet.hadoop.example.GroupReadSupport; 38 | import parquet.hadoop.metadata.ParquetMetadata; 39 | import parquet.io.ColumnIOFactory; 40 | import parquet.io.MessageColumnIO; 41 | import parquet.io.RecordReader; 42 | import parquet.schema.MessageType; 43 | import parquet.schema.MessageTypeParser; 44 | 45 | public class ConvertUtils { 46 | 47 | private static final Log LOG = Log.getLog(ConvertUtils.class); 48 | 49 | public static final String CSV_DELIMITER= "|"; 50 | 51 | private static String readFile(String path) throws IOException { 52 | BufferedReader reader = new BufferedReader(new FileReader(path)); 53 | StringBuilder stringBuilder = new StringBuilder(); 54 | 55 | try { 56 | String line = null; 57 | String ls = System.getProperty("line.separator"); 58 | 59 | while ((line = reader.readLine()) != null ) { 60 | stringBuilder.append(line); 61 | stringBuilder.append(ls); 62 | } 63 | } finally { 64 | Utils.closeQuietly(reader); 65 | } 66 | 67 | return stringBuilder.toString(); 68 | } 69 | 70 | public static String getSchema(File csvFile) throws IOException { 71 | String fileName = csvFile.getName().substring( 72 | 0, csvFile.getName().length() - ".csv".length()) + ".schema"; 73 | File schemaFile = new File(csvFile.getParentFile(), fileName); 74 | return readFile(schemaFile.getAbsolutePath()); 75 | } 76 | 77 | public static void convertCsvToParquet(File csvFile, File outputParquetFile) throws IOException { 78 | convertCsvToParquet(csvFile, outputParquetFile, false); 79 | } 80 | 81 | public static void convertCsvToParquet(File csvFile, File outputParquetFile, boolean enableDictionary) throws IOException { 82 | LOG.info("Converting " + csvFile.getName() + " to " + outputParquetFile.getName()); 83 | String rawSchema = getSchema(csvFile); 84 | if(outputParquetFile.exists()) { 85 | throw new IOException("Output file " + outputParquetFile.getAbsolutePath() + 86 | " already exists"); 87 | } 88 | 89 | Path path = new Path(outputParquetFile.toURI()); 90 | 91 | MessageType schema = MessageTypeParser.parseMessageType(rawSchema); 92 | CsvParquetWriter writer = new CsvParquetWriter(path, schema, enableDictionary); 93 | 94 | BufferedReader br = new BufferedReader(new FileReader(csvFile)); 95 | String line; 96 | int lineNumber = 0; 97 | try { 98 | while ((line = br.readLine()) != null) { 99 | String[] fields = line.split(Pattern.quote(CSV_DELIMITER)); 100 | writer.write(Arrays.asList(fields)); 101 | ++lineNumber; 102 | } 103 | 104 | writer.close(); 105 | } finally { 106 | LOG.info("Number of lines: " + lineNumber); 107 | Utils.closeQuietly(br); 108 | } 109 | } 110 | 111 | public static void convertParquetToCSV(File parquetFile, File csvOutputFile) throws IOException { 112 | Preconditions.checkArgument(parquetFile.getName().endsWith(".parquet"), 113 | "parquet file should have .parquet extension"); 114 | Preconditions.checkArgument(csvOutputFile.getName().endsWith(".csv"), 115 | "csv file should have .csv extension"); 116 | Preconditions.checkArgument(!csvOutputFile.exists(), 117 | "Output file " + csvOutputFile.getAbsolutePath() + " already exists"); 118 | 119 | LOG.info("Converting " + parquetFile.getName() + " to " + csvOutputFile.getName()); 120 | 121 | 122 | Path parquetFilePath = new Path(parquetFile.toURI()); 123 | 124 | Configuration configuration = new Configuration(true); 125 | 126 | GroupReadSupport readSupport = new GroupReadSupport(); 127 | ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, parquetFilePath); 128 | MessageType schema = readFooter.getFileMetaData().getSchema(); 129 | 130 | readSupport.init(configuration, null, schema); 131 | BufferedWriter w = new BufferedWriter(new FileWriter(csvOutputFile)); 132 | ParquetReader reader = new ParquetReader(parquetFilePath, readSupport); 133 | try{ 134 | Group g = null; 135 | while( (g = reader.read())!= null) { 136 | writeGroup(w, g, schema); 137 | } 138 | reader.close(); 139 | } 140 | finally { 141 | Utils.closeQuietly(w); 142 | } 143 | } 144 | 145 | private static void writeGroup(BufferedWriter w, Group g, MessageType schema) 146 | throws IOException{ 147 | for (int j = 0; j < schema.getFieldCount(); j++) { 148 | if (j > 0) { 149 | w.write(CSV_DELIMITER); 150 | } 151 | String valueToString = g.getValueToString(j, 0); 152 | w.write(valueToString); 153 | } 154 | w.write('\n'); 155 | } 156 | 157 | @Deprecated 158 | public static void convertParquetToCSVEx(File parquetFile, File csvOutputFile) throws IOException { 159 | Preconditions.checkArgument(parquetFile.getName().endsWith(".parquet"), 160 | "parquet file should have .parquet extension"); 161 | Preconditions.checkArgument(csvOutputFile.getName().endsWith(".csv"), 162 | "csv file should have .csv extension"); 163 | Preconditions.checkArgument(!csvOutputFile.exists(), 164 | "Output file " + csvOutputFile.getAbsolutePath() + " already exists"); 165 | 166 | LOG.info("Converting " + parquetFile.getName() + " to " + csvOutputFile.getName()); 167 | 168 | Path parquetFilePath = new Path(parquetFile.toURI()); 169 | 170 | Configuration configuration = new Configuration(true); 171 | 172 | // TODO Following can be changed by using ParquetReader instead of ParquetFileReader 173 | ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, parquetFilePath); 174 | MessageType schema = readFooter.getFileMetaData().getSchema(); 175 | ParquetFileReader parquetFileReader = new ParquetFileReader( 176 | configuration, parquetFilePath, readFooter.getBlocks(), schema.getColumns()); 177 | BufferedWriter w = new BufferedWriter(new FileWriter(csvOutputFile)); 178 | PageReadStore pages = null; 179 | try { 180 | while (null != (pages = parquetFileReader.readNextRowGroup())) { 181 | final long rows = pages.getRowCount(); 182 | LOG.info("Number of rows: " + rows); 183 | 184 | final MessageColumnIO columnIO = new ColumnIOFactory().getColumnIO(schema); 185 | final RecordReader recordReader = columnIO.getRecordReader(pages, new GroupRecordConverter(schema)); 186 | for (int i = 0; i < rows; i++) { 187 | final Group g = recordReader.read(); 188 | writeGroup(w, g, schema); 189 | } 190 | } 191 | } finally { 192 | Utils.closeQuietly(parquetFileReader); 193 | Utils.closeQuietly(w); 194 | } 195 | } 196 | 197 | } 198 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/CsvParquetWriter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import java.io.IOException; 19 | import java.util.List; 20 | 21 | import org.apache.hadoop.fs.Path; 22 | 23 | import parquet.hadoop.ParquetWriter; 24 | import parquet.hadoop.api.WriteSupport; 25 | import parquet.hadoop.metadata.CompressionCodecName; 26 | import parquet.schema.MessageType; 27 | 28 | public class CsvParquetWriter extends ParquetWriter> { 29 | 30 | public CsvParquetWriter(Path file, MessageType schema) throws IOException { 31 | this(file, schema, false); 32 | } 33 | 34 | public CsvParquetWriter(Path file, MessageType schema, boolean enableDictionary) throws IOException { 35 | this(file, schema, CompressionCodecName.UNCOMPRESSED, enableDictionary); 36 | } 37 | 38 | public CsvParquetWriter(Path file, MessageType schema, CompressionCodecName codecName, boolean enableDictionary) throws IOException { 39 | super(file, (WriteSupport>) new CsvWriteSupport(schema), codecName, DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE, enableDictionary, false); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/CsvWriteSupport.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import java.util.HashMap; 19 | import java.util.List; 20 | 21 | import org.apache.hadoop.conf.Configuration; 22 | 23 | import parquet.column.ColumnDescriptor; 24 | import parquet.hadoop.api.WriteSupport; 25 | import parquet.io.ParquetEncodingException; 26 | import parquet.io.api.Binary; 27 | import parquet.io.api.RecordConsumer; 28 | import parquet.schema.MessageType; 29 | 30 | public class CsvWriteSupport extends WriteSupport> { 31 | MessageType schema; 32 | RecordConsumer recordConsumer; 33 | List cols; 34 | 35 | // TODO: support specifying encodings and compression 36 | public CsvWriteSupport(MessageType schema) { 37 | this.schema = schema; 38 | this.cols = schema.getColumns(); 39 | } 40 | 41 | @Override 42 | public WriteContext init(Configuration config) { 43 | return new WriteContext(schema, new HashMap()); 44 | } 45 | 46 | @Override 47 | public void prepareForWrite(RecordConsumer r) { 48 | recordConsumer = r; 49 | } 50 | 51 | @Override 52 | public void write(List values) { 53 | if (values.size() != cols.size()) { 54 | throw new ParquetEncodingException("Invalid input data. Expecting " + 55 | cols.size() + " columns. Input had " + values.size() + " columns (" + cols + ") : " + values); 56 | } 57 | 58 | recordConsumer.startMessage(); 59 | for (int i = 0; i < cols.size(); ++i) { 60 | String val = values.get(i); 61 | // val.length() == 0 indicates a NULL value. 62 | if (val.length() > 0) { 63 | recordConsumer.startField(cols.get(i).getPath()[0], i); 64 | switch (cols.get(i).getType()) { 65 | case BOOLEAN: 66 | recordConsumer.addBoolean(Boolean.parseBoolean(val)); 67 | break; 68 | case FLOAT: 69 | recordConsumer.addFloat(Float.parseFloat(val)); 70 | break; 71 | case DOUBLE: 72 | recordConsumer.addDouble(Double.parseDouble(val)); 73 | break; 74 | case INT32: 75 | recordConsumer.addInteger(Integer.parseInt(val)); 76 | break; 77 | case INT64: 78 | recordConsumer.addLong(Long.parseLong(val)); 79 | break; 80 | case BINARY: 81 | recordConsumer.addBinary(stringToBinary(val)); 82 | break; 83 | default: 84 | throw new ParquetEncodingException( 85 | "Unsupported column type: " + cols.get(i).getType()); 86 | } 87 | recordConsumer.endField(cols.get(i).getPath()[0], i); 88 | } 89 | } 90 | recordConsumer.endMessage(); 91 | } 92 | 93 | private Binary stringToBinary(Object value) { 94 | return Binary.fromString(value.toString()); 95 | } 96 | } 97 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/TestBackwardsCompatibility.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import java.io.File; 19 | import java.io.IOException; 20 | 21 | import org.junit.Test; 22 | 23 | import parquet.Log; 24 | 25 | /** 26 | * This tests compatibility of parquet format (written by java code) 27 | * from older versions of parquet with the current version. 28 | * 29 | * Parquet files for previous versions are assumed to be generated under 30 | * under $PROJECT_HOME/parquet-compat-$version/target/parquet/ 31 | * If files are not present, a WARNing is generated. 32 | * 33 | * @author amokashi 34 | * 35 | */ 36 | public class TestBackwardsCompatibility { 37 | 38 | private static final Log LOG = Log.getLog(TestBackwardsCompatibility.class); 39 | 40 | @Test 41 | public void testReadWriteCompatibility() throws IOException { 42 | File[] csvFiles = Utils.getAllOriginalCSVFiles(); 43 | for (File csvFile : csvFiles) { 44 | String filename = Utils.getFileNamePrefix(csvFile); 45 | 46 | // With no dictionary - default 47 | File parquetTestFile = Utils.getParquetOutputFile(filename, "plain", true); 48 | ConvertUtils.convertCsvToParquet(csvFile, parquetTestFile); 49 | File csvTestFile = Utils.getCsvTestFile(filename, "plain", true); 50 | ConvertUtils.convertParquetToCSV(parquetTestFile, csvTestFile); 51 | 52 | Utils.verify(csvFile, csvTestFile); 53 | 54 | // With dictionary encoding 55 | parquetTestFile = Utils.getParquetOutputFile(filename, "dict", true); 56 | ConvertUtils.convertCsvToParquet(csvFile, parquetTestFile, true); 57 | csvTestFile = Utils.getCsvTestFile(filename, "dict", true); 58 | ConvertUtils.convertParquetToCSV(parquetTestFile, csvTestFile); 59 | 60 | Utils.verify(csvFile, csvTestFile); 61 | } 62 | } 63 | 64 | @Test 65 | public void testParquetBackwardsCompatibility() throws IOException { 66 | // read all versions of parquet files and convert them into csv 67 | // diff the csvs with original csvs 68 | File[] originalCsvFiles = Utils.getAllOriginalCSVFiles(); 69 | String[] compatibleVersions = Utils.getAllPreviousVersionDirs(); 70 | 71 | for (String version : compatibleVersions) { 72 | LOG.info("Testing compatibility with " + version); 73 | for(File originalCsvFile : originalCsvFiles) { 74 | 75 | String prefix = Utils.getFileNamePrefix(originalCsvFile); 76 | File versionParquetFile = Utils.getParquetFile(prefix, version, "plain", true); 77 | File csvVersionedTestFile = Utils.getCsvTestFile(prefix, version, true); 78 | 79 | ConvertUtils.convertParquetToCSV(versionParquetFile, csvVersionedTestFile); 80 | 81 | Utils.verify(originalCsvFile, csvVersionedTestFile); 82 | } 83 | } 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/TestImpalaCompatibility.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import java.io.File; 19 | import java.io.IOException; 20 | 21 | import org.junit.Test; 22 | 23 | import parquet.Log; 24 | 25 | public class TestImpalaCompatibility { 26 | private static final Log LOG = Log.getLog(TestImpalaCompatibility.class); 27 | 28 | @Test 29 | public void testReadFromImpala() throws IOException { 30 | 31 | File[] originalCsvFiles = Utils.getAllOriginalCSVFiles(); 32 | String[] impalaVersions = Utils.getImpalaDirectories(); 33 | LOG.info("Testing compatibility in reading files written by impala"); 34 | 35 | for(String impalaVersion : impalaVersions) { 36 | for(File originalCsv : originalCsvFiles) { 37 | String prefix = Utils.getFileNamePrefix(originalCsv); 38 | File parquetFile = null; 39 | try { 40 | parquetFile = Utils.getParquetImpalaFile(prefix, impalaVersion); 41 | } catch (Exception e){continue;} 42 | File csvOutputFile = Utils.getCsvTestFile(prefix, "impala", true); 43 | ConvertUtils.convertParquetToCSV(parquetFile, csvOutputFile); 44 | Utils.verify(originalCsv, csvOutputFile, false); 45 | } 46 | } 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/TestPerfRegression.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import java.io.File; 19 | import java.io.IOException; 20 | 21 | import static org.junit.Assert.assertTrue; 22 | 23 | import org.junit.Test; 24 | 25 | import parquet.Log; 26 | 27 | public class TestPerfRegression { 28 | private static final Log LOG = Log.getLog(TestPerfRegression.class); 29 | 30 | private static final int fileSizeMegs = 1000; 31 | 32 | @Test 33 | public void testWritePerf() throws IOException { 34 | // With no dictionary - default 35 | File csvTestFile = Utils.createTestFile(fileSizeMegs); 36 | 37 | File parquetTestFile = Utils.getParquetOutputFile("perf", String.valueOf(fileSizeMegs), true); 38 | long startTime = System.currentTimeMillis(); 39 | ConvertUtils.convertCsvToParquet(csvTestFile, parquetTestFile); 40 | long endTime = System.currentTimeMillis(); 41 | 42 | long totalTime = (endTime - startTime); 43 | LOG.info("Write Time: " + totalTime ); 44 | 45 | Utils.writePerfResult("write", totalTime); 46 | 47 | LOG.info("Time taken to write " + fileSizeMegs + " sized csv file : " + fileSizeMegs); 48 | 49 | assertTrue(totalTime < 60000); 50 | 51 | // It should not be slower than previous versions 52 | for(String version : Utils.getAllPreviousVersionDirs()) { 53 | long totalTimeForVersion = Utils.readPerfResult(version, "write"); 54 | LOG.info("Time taken to write with version "+ version + ": " + totalTimeForVersion); 55 | assertTrue(totalTime < 1.1 * totalTimeForVersion); 56 | } 57 | } 58 | 59 | @Test 60 | public void testReadPerf() throws IOException { 61 | File parquetTestFile = Utils.getParquetOutputFile("perf", String.valueOf(fileSizeMegs), false); 62 | if(!parquetTestFile.exists()) { 63 | throw new IOException("File "+ parquetTestFile.getName() + " does not exists, Run testWritePerf"); 64 | } 65 | File csvTestFile = Utils.getCsvTestFile("perf", String.valueOf(fileSizeMegs), true); 66 | long startTime = System.currentTimeMillis(); 67 | ConvertUtils.convertParquetToCSV(parquetTestFile, csvTestFile ); 68 | long endTime = System.currentTimeMillis(); 69 | 70 | long totalTime = (endTime - startTime); 71 | LOG.info("Read Time: " + totalTime ); 72 | 73 | Utils.writePerfResult("read", totalTime); 74 | 75 | assertTrue(totalTime < 45000); 76 | 77 | // It should not be slower than previous versions 78 | for(String version : Utils.getAllPreviousVersionDirs()) { 79 | long totalTimeForVersion = Utils.readPerfResult(version, "read"); 80 | LOG.info("Time taken to read with version "+ version + ": " + totalTimeForVersion); 81 | assertTrue(totalTime < 1.1 * totalTimeForVersion); 82 | } 83 | } 84 | 85 | } 86 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/TestUtils.java: -------------------------------------------------------------------------------- 1 | package parquet.compat.test; 2 | 3 | import junit.framework.Assert; 4 | 5 | import org.junit.Test; 6 | 7 | public class TestUtils { 8 | 9 | @Test 10 | public void testVersionComparator() { 11 | Utils.Version v1 = new Utils.Version("1.0.0"); 12 | Utils.Version v2 = new Utils.Version("1.0.1"); 13 | 14 | Assert.assertTrue(v1.compareMajorMinor(v2)==0); 15 | Assert.assertTrue(v1.compareTo(v2) < 0); 16 | 17 | v2 = new Utils.Version("1.1.0"); 18 | 19 | Assert.assertTrue(v1.compareMajorMinor(v2) < 0); 20 | Assert.assertTrue(v1.compareTo(v2) < 0); 21 | 22 | v2 = new Utils.Version("1.0.0-SNAPSHOT"); 23 | 24 | Assert.assertTrue(v1.compareTo(v2) > 0); 25 | 26 | } 27 | 28 | } 29 | -------------------------------------------------------------------------------- /parquet-compat/src/test/java/parquet/compat/test/Utils.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2012 Twitter, Inc. 3 | * 4 | * Licensed under the Apache License, Version 2.0 (the "License"); 5 | * you may not use this file except in compliance with the License. 6 | * You may obtain a copy of the License at 7 | * 8 | * http://www.apache.org/licenses/LICENSE-2.0 9 | * 10 | * Unless required by applicable law or agreed to in writing, software 11 | * distributed under the License is distributed on an "AS IS" BASIS, 12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | * See the License for the specific language governing permissions and 14 | * limitations under the License. 15 | */ 16 | package parquet.compat.test; 17 | 18 | import static org.junit.Assert.assertEquals; 19 | import static org.junit.Assert.assertNull; 20 | 21 | import java.io.BufferedInputStream; 22 | import java.io.BufferedOutputStream; 23 | import java.io.BufferedReader; 24 | import java.io.Closeable; 25 | import java.io.File; 26 | import java.io.FileInputStream; 27 | import java.io.FileOutputStream; 28 | import java.io.FileReader; 29 | import java.io.FileWriter; 30 | import java.io.FilenameFilter; 31 | import java.io.IOException; 32 | import java.io.InputStream; 33 | import java.io.OutputStream; 34 | import java.io.PrintWriter; 35 | import java.util.ArrayList; 36 | import java.util.Collections; 37 | import java.util.List; 38 | 39 | import org.apache.commons.io.FileUtils; 40 | import org.apache.commons.io.IOUtils; 41 | import org.apache.hadoop.io.file.tfile.Utils.Version; 42 | 43 | import parquet.Log; 44 | 45 | public class Utils { 46 | 47 | private static final Log LOG = Log.getLog(Utils.class); 48 | 49 | public static void closeQuietly(Closeable res) { 50 | try { 51 | if(res != null) { 52 | res.close(); 53 | } 54 | } catch (IOException ioe) { 55 | LOG.warn("Exception closing reader " + res + ": " + ioe.getMessage()); 56 | } 57 | } 58 | 59 | public static void writePerfResult(String module, long millis) throws IOException{ 60 | PrintWriter writer = null; 61 | try { 62 | File outputFile = new File("target/test/perftime." + module + ".txt"); 63 | outputFile.delete(); 64 | writer = new PrintWriter(outputFile); 65 | writer.write(String.valueOf(millis)); 66 | } finally { 67 | closeQuietly(writer); 68 | } 69 | } 70 | 71 | public static long readPerfResult(String version, String module) throws IOException { 72 | BufferedReader reader = null; 73 | try { 74 | File inFile = new File("../" + version + "/target/test/perftime." + module + ".txt"); 75 | reader = new BufferedReader(new FileReader(inFile)); 76 | return Long.parseLong(reader.readLine()); 77 | } finally { 78 | closeQuietly(reader); 79 | } 80 | } 81 | 82 | public static File createTestFile(long largerThanMB) throws IOException { 83 | File outputFile = new File("target/test/csv/perftest.csv"); 84 | if(outputFile.exists()) { 85 | return outputFile; 86 | } 87 | File toCopy = new File("../parquet-testdata/tpch/customer.csv"); 88 | FileUtils.copyFile(new File("../parquet-testdata/tpch/customer.schema"), new File("target/test/csv/perftest.schema")); 89 | 90 | OutputStream output = null; 91 | InputStream input = null; 92 | 93 | try { 94 | output = new BufferedOutputStream(new FileOutputStream(outputFile, true)); 95 | input = new BufferedInputStream(new FileInputStream(toCopy)); 96 | input.mark(Integer.MAX_VALUE); 97 | while(outputFile.length() <= largerThanMB * 1024 * 1024) { 98 | //appendFile(output, toCopy); 99 | IOUtils.copy(input, output); 100 | input.reset(); 101 | } 102 | } finally { 103 | closeQuietly(input); 104 | closeQuietly(output); 105 | } 106 | 107 | return outputFile; 108 | } 109 | 110 | public static File[] getAllOriginalCSVFiles() { 111 | File baseDir = new File("../parquet-testdata/tpch"); 112 | final File[] csvFiles = baseDir.listFiles(new FilenameFilter() { 113 | public boolean accept(File dir, String name) { 114 | return name.endsWith(".csv"); 115 | } 116 | }); 117 | return csvFiles; 118 | } 119 | 120 | public static String[] getAllPreviousVersionDirs() throws IOException { 121 | File baseDir = new File(".."); 122 | final String currentVersion = getCurrentVersion(); 123 | final String[] versions = baseDir.list(new FilenameFilter() { 124 | public boolean accept(File dir, String name) { 125 | return name.startsWith("parquet-compat-") 126 | && new Version(name.replace("parquet-compat-", "")).compareTo(new Version(currentVersion)) < 0; 127 | } 128 | }); 129 | return versions; 130 | } 131 | 132 | static class Version implements Comparable { 133 | int major; 134 | int minor; 135 | int minorminor; 136 | String tag; 137 | 138 | Version(String versionStr) { 139 | String[] versions = versionStr.split("\\."); 140 | int size = versions.length; 141 | if (size > 0) { 142 | this.major = Integer.parseInt(versions[0]); 143 | } 144 | if (size > 1) { 145 | this.minor = Integer.parseInt(versions[1]); 146 | } 147 | if (size > 2) { 148 | if(versions[2].contains("-")) { 149 | String[] minorMin = versions[2].split("-"); 150 | this.minorminor = Integer.parseInt(minorMin[0]); 151 | this.tag = minorMin[1]; 152 | } else { 153 | this.minorminor = Integer.parseInt(versions[2]); 154 | } 155 | } 156 | if(size == 4) { 157 | this.tag = versions[3]; 158 | } 159 | if (size > 4) { 160 | throw new RuntimeException("Illegal version number " + versionStr); 161 | } 162 | } 163 | 164 | public int compareMajorMinor(Version o) { 165 | return ComparisonChain. 166 | start(). 167 | compare(major, o.major). 168 | compare(minor, o.minor). 169 | result(); 170 | } 171 | 172 | @Override 173 | public int compareTo(Version o) { 174 | return ComparisonChain. 175 | start(). 176 | compare(major, o.major). 177 | compare(minor, o.minor). 178 | compare(minorminor, o.minorminor). 179 | compare(tag, o.tag). 180 | result(); 181 | } 182 | 183 | // Very basic implementation of comparisonchain 184 | private static class ComparisonChain { 185 | int result = 0; 186 | private ComparisonChain(int result) { 187 | this.result = result; 188 | } 189 | static ComparisonChain start() { 190 | return new ComparisonChain(0); 191 | } 192 | ComparisonChain compare(String a, String b) { 193 | if (result != 0) { 194 | return this; 195 | } 196 | if(b == null) { 197 | if (a!= null) result=1; 198 | else result= 0; 199 | } else if (a == null) { 200 | result=1; 201 | } else if (result == 0) { 202 | result = a.compareTo(b); 203 | } 204 | return this; 205 | } 206 | ComparisonChain compare(int a, int b) { 207 | if (result == 0) { 208 | result = Integer.compare(a, b); 209 | } 210 | return this; 211 | } 212 | int result() { 213 | return result; 214 | } 215 | } 216 | } 217 | 218 | 219 | public static File getParquetOutputFile(String name, String module, boolean deleteIfExists) { 220 | File outputFile = new File("target/parquet/", getParquetFileName(name, module)); 221 | outputFile.getParentFile().mkdirs(); 222 | if(deleteIfExists) { 223 | outputFile.delete(); 224 | } 225 | return outputFile; 226 | } 227 | 228 | private static String getParquetFileName (String name, String module) { 229 | return name + (module != null ? "." + module : "") + ".parquet"; 230 | } 231 | 232 | public static File getParquetFile(String name, String version, String module, boolean failIfNotExist) 233 | throws IOException { 234 | File parquetFile = new File("../"+version+"/target/parquet/", getParquetFileName(name, module)); 235 | parquetFile.getParentFile().mkdirs(); 236 | if(!parquetFile.exists()) { 237 | String msg = "File " + parquetFile.getAbsolutePath() + " does not exist"; 238 | if(failIfNotExist) { 239 | throw new IOException(msg); 240 | } 241 | LOG.warn(msg); 242 | } 243 | return parquetFile; 244 | } 245 | 246 | private static String getCurrentVersion() throws IOException { 247 | return new File(".").getCanonicalFile().getName().replace("parquet-compat-", ""); 248 | } 249 | 250 | public static String[] getImpalaDirectories() throws IOException { 251 | File baseDir = new File("../parquet-testdata/impala"); 252 | final String currentVersion = getCurrentVersion(); 253 | final String[] impalaVersions = baseDir.list(new FilenameFilter() { 254 | public boolean accept(File dir, String name) { 255 | if (name.startsWith(".")) { 256 | return false; 257 | } 258 | if (name.contains("-")) { 259 | name = name.split("-")[0]; 260 | } 261 | return new Version(name).compareMajorMinor(new Version(currentVersion)) == 0; 262 | } 263 | }); 264 | return impalaVersions; 265 | } 266 | 267 | public static File getParquetImpalaFile(String name, String impalaVersion) throws IOException { 268 | String fileName = name + ".impala.parquet"; 269 | File parquetFile = new File("../parquet-testdata/impala/" + impalaVersion, fileName); 270 | if(!parquetFile.exists()) { 271 | throw new IOException("File " + fileName + " does not exist"); 272 | } 273 | return parquetFile; 274 | } 275 | 276 | public static String getFileNamePrefix(File file) { 277 | return file.getName().substring(0, file.getName().indexOf(".")); 278 | } 279 | 280 | public static File getCsvTestFile(String name, String module, boolean deleteIfExists) { 281 | File outputFile = new File("target/test/csv/", 282 | name + (module != null ? "." + module : "") + ".csv"); 283 | outputFile.getParentFile().mkdirs(); 284 | if(deleteIfExists) { 285 | outputFile.delete(); 286 | } 287 | return outputFile; 288 | } 289 | 290 | public static File getParquetTestFile(String name, String module, boolean deleteIfExists) { 291 | File outputFile = new File("target/test/parquet/", 292 | name + (module != null ? "." + module : "") + ".csv"); 293 | outputFile.getParentFile().mkdirs(); 294 | if(deleteIfExists) { 295 | outputFile.delete(); 296 | } 297 | return outputFile; 298 | } 299 | 300 | public static void verify(File expectedCsvFile, File outputCsvFile) throws IOException { 301 | BufferedReader expected = null; 302 | BufferedReader out = null; 303 | try { 304 | expected = new BufferedReader(new FileReader(expectedCsvFile)); 305 | out = new BufferedReader(new FileReader(outputCsvFile)); 306 | String lineIn; 307 | String lineOut = null; 308 | int lineNumber = 0; 309 | while ((lineIn = expected.readLine()) != null && (lineOut = out.readLine()) != null) { 310 | ++ lineNumber; 311 | lineOut = lineOut.substring(lineOut.indexOf("\t") + 1); 312 | assertEquals("line " + lineNumber, lineIn, lineOut); 313 | } 314 | assertNull("line " + lineNumber, lineIn); 315 | assertNull("line " + lineNumber, out.readLine()); 316 | } finally { 317 | Utils.closeQuietly(expected); 318 | Utils.closeQuietly(out); 319 | } 320 | } 321 | 322 | public static void verify(File expectedCsvFile, File outputCsvFile, boolean orderMatters) throws IOException { 323 | if(!orderMatters) { 324 | // sort the files before diff'ing them 325 | expectedCsvFile = sortFile(expectedCsvFile); 326 | outputCsvFile = sortFile(outputCsvFile); 327 | } 328 | verify(expectedCsvFile, outputCsvFile); 329 | } 330 | 331 | private static File sortFile(File inFile) throws IOException { 332 | File sortedFile = new File(inFile.getAbsolutePath().concat(".sorted")); 333 | BufferedReader reader = new BufferedReader(new FileReader(inFile)); 334 | PrintWriter out = new PrintWriter(new FileWriter(sortedFile)); 335 | 336 | try { 337 | String inputLine; 338 | List lineList = new ArrayList(); 339 | while ((inputLine = reader.readLine()) != null) { 340 | lineList.add(inputLine); 341 | } 342 | Collections.sort(lineList); 343 | 344 | for (String outputLine : lineList) { 345 | out.println(outputLine); 346 | } 347 | out.flush(); 348 | } finally { 349 | closeQuietly(reader); 350 | closeQuietly(out); 351 | } 352 | return sortedFile; 353 | } 354 | } 355 | -------------------------------------------------------------------------------- /parquet-testdata/impala/1.1.1-GZIP/customer.impala.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Parquet/parquet-compatibility/2b47eac447c7a4a88247651a4065984db7b247ff/parquet-testdata/impala/1.1.1-GZIP/customer.impala.parquet -------------------------------------------------------------------------------- /parquet-testdata/impala/1.1.1-GZIP/nation.impala.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Parquet/parquet-compatibility/2b47eac447c7a4a88247651a4065984db7b247ff/parquet-testdata/impala/1.1.1-GZIP/nation.impala.parquet -------------------------------------------------------------------------------- /parquet-testdata/impala/1.1.1-NONE/customer.impala.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Parquet/parquet-compatibility/2b47eac447c7a4a88247651a4065984db7b247ff/parquet-testdata/impala/1.1.1-NONE/customer.impala.parquet -------------------------------------------------------------------------------- /parquet-testdata/impala/1.1.1-NONE/nation.impala.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Parquet/parquet-compatibility/2b47eac447c7a4a88247651a4065984db7b247ff/parquet-testdata/impala/1.1.1-NONE/nation.impala.parquet -------------------------------------------------------------------------------- /parquet-testdata/impala/1.1.1-SNAPPY/customer.impala.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Parquet/parquet-compatibility/2b47eac447c7a4a88247651a4065984db7b247ff/parquet-testdata/impala/1.1.1-SNAPPY/customer.impala.parquet -------------------------------------------------------------------------------- /parquet-testdata/impala/1.1.1-SNAPPY/nation.impala.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/Parquet/parquet-compatibility/2b47eac447c7a4a88247651a4065984db7b247ff/parquet-testdata/impala/1.1.1-SNAPPY/nation.impala.parquet -------------------------------------------------------------------------------- /parquet-testdata/tpch/customer.schema: -------------------------------------------------------------------------------- 1 | message m { 2 | optional int64 cust_key; 3 | optional binary name; 4 | optional binary address; 5 | optional int32 nation_key; 6 | optional binary phone; 7 | optional double acctbal; 8 | optional binary mktsegment; 9 | optional binary comment_col; 10 | } 11 | -------------------------------------------------------------------------------- /parquet-testdata/tpch/customer.sql: -------------------------------------------------------------------------------- 1 | -- assume customer.csv is copied to /user/cloudera/customer 2 | -- 1.0 of impala doesn't support load data local inpath 3 | -- ToDo change this to use load data local inpath for imapala 1.1 4 | create external table customer ( 5 | cust_key bigint, 6 | name string, 7 | address string, 8 | nation_key int, 9 | phone string, 10 | acctbal double, 11 | mktsegment string, 12 | comment_col string 13 | ) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' 14 | location '/user/cloudera/customer'; 15 | 16 | create table customer_parquet like customer stored as parquetfile; 17 | 18 | insert overwrite table customer_parquet select * from customer; 19 | -------------------------------------------------------------------------------- /parquet-testdata/tpch/nation.csv: -------------------------------------------------------------------------------- 1 | 0|ALGERIA|0| haggle. carefully final deposits detect slyly agai 2 | 1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon 3 | 2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special 4 | 3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold 5 | 4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d 6 | 5|ETHIOPIA|0|ven packages wake quickly. regu 7 | 6|FRANCE|3|refully final requests. regular, ironi 8 | 7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco 9 | 8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun 10 | 9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull 11 | 10|IRAN|4|efully alongside of the slyly final dependencies. 12 | 11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula 13 | 12|JAPAN|2|ously. final, express gifts cajole a 14 | 13|JORDAN|4|ic deposits are blithely about the carefully regular pa 15 | 14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t 16 | 15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets? 17 | 16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r 18 | 17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun 19 | 18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos 20 | 19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account 21 | 20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely 22 | 21|VIETNAM|2|hely enticingly express accounts. even, final 23 | 22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint 24 | 23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull 25 | 24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be 26 | -------------------------------------------------------------------------------- /parquet-testdata/tpch/nation.schema: -------------------------------------------------------------------------------- 1 | message m { 2 | optional int32 nation_key; 3 | optional binary name; 4 | optional int32 region_key; 5 | optional binary comment_col; 6 | } 7 | -------------------------------------------------------------------------------- /parquet-testdata/tpch/nation.sql: -------------------------------------------------------------------------------- 1 | create external table nation ( 2 | nation_key int, 3 | name string, 4 | region_key int, 5 | comment_col string 6 | ) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' 7 | location '/user/cloudera/nation'; 8 | 9 | create table nation_parquet like nation stored as parquetfile; 10 | 11 | insert overwrite table nation_parquet select * from nation; 12 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 3 | 4.0.0 4 | 5 | parquet 6 | parquet-compatibility 7 | 1.0.0 8 | pom 9 | 10 | Parquet Compatibility 11 | https://github.com/Parquet/parquet-compatibility 12 | 13 | 14 | UTF-8 15 | 16 | 17 | 18 | parquet-compat-1.0.0 19 | parquet-compat-1.1.0 20 | parquet-compat-1.2.0 21 | 22 | 23 | 24 | 25 | junit 26 | junit 27 | 4.10 28 | test 29 | 30 | 31 | 32 | --------------------------------------------------------------------------------