-
Notifications
You must be signed in to change notification settings - Fork 1.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Support for storing result sets in Parquet format
- Loading branch information
1 parent
4c17598
commit 197e9a3
Showing
13 changed files
with
351 additions
and
36 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
122 changes: 122 additions & 0 deletions
122
...kis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetReader.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,122 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.linkis.storage.resultset; | ||
|
||
import org.apache.linkis.common.io.FsPath; | ||
import org.apache.linkis.common.io.MetaData; | ||
import org.apache.linkis.common.io.Record; | ||
import org.apache.linkis.common.io.resultset.ResultSet; | ||
import org.apache.linkis.storage.domain.Column; | ||
import org.apache.linkis.storage.domain.DataType; | ||
import org.apache.linkis.storage.resultset.table.TableMetaData; | ||
import org.apache.linkis.storage.resultset.table.TableRecord; | ||
|
||
import org.apache.avro.Schema; | ||
import org.apache.avro.generic.GenericRecord; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.parquet.avro.AvroParquetReader; | ||
import org.apache.parquet.hadoop.ParquetReader; | ||
|
||
import java.io.IOException; | ||
import java.io.InputStream; | ||
import java.util.ArrayList; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class ParquetResultSetReader<K extends MetaData, V extends Record> | ||
extends StorageResultSetReader { | ||
|
||
private static final Logger logger = LoggerFactory.getLogger(ParquetResultSetReader.class); | ||
|
||
private FsPath fsPath; | ||
|
||
private ParquetReader<GenericRecord> parquetReader; | ||
|
||
private GenericRecord record; | ||
|
||
public ParquetResultSetReader(ResultSet resultSet, InputStream inputStream, FsPath fsPath) | ||
throws IOException { | ||
super(resultSet, inputStream); | ||
this.fsPath = fsPath; | ||
this.parquetReader = | ||
AvroParquetReader.<GenericRecord>builder(new Path(fsPath.getPath())).build(); | ||
this.record = parquetReader.read(); | ||
} | ||
|
||
@Override | ||
public MetaData getMetaData() { | ||
if (metaData == null) { | ||
try { | ||
List<Schema.Field> fields = record.getSchema().getFields(); | ||
List<Column> columnList = | ||
fields.stream() | ||
.map( | ||
field -> | ||
new Column( | ||
field.name(), | ||
DataType.toDataType(field.schema().getType().getName()), | ||
"")) | ||
.collect(Collectors.toList()); | ||
|
||
metaData = new TableMetaData(columnList.toArray(new Column[0])); | ||
} catch (Exception e) { | ||
throw new RuntimeException("Failed to read parquet schema", e); | ||
} | ||
} | ||
return metaData; | ||
} | ||
|
||
@Override | ||
public boolean hasNext() throws IOException { | ||
if (metaData == null) getMetaData(); | ||
if (record == null) return false; | ||
ArrayList<Object> resultList = new ArrayList<>(); | ||
TableMetaData tableMetaData = (TableMetaData) metaData; | ||
int length = tableMetaData.getColumns().length; | ||
for (int i = 0; i < length; i++) { | ||
resultList.add(record.get(i)); | ||
} | ||
row = new TableRecord(resultList.toArray(new Object[0])); | ||
if (row == null) return false; | ||
return record != null; | ||
} | ||
|
||
@Override | ||
public Record getRecord() { | ||
if (metaData == null) throw new RuntimeException("Must read metadata first(必须先读取metadata)"); | ||
if (row == null) { | ||
throw new RuntimeException( | ||
"Can't get the value of the field, maybe the IO stream has been read or has been closed!(拿不到字段的值,也许IO流已读取完毕或已被关闭!)"); | ||
} | ||
try { | ||
this.record = parquetReader.read(); | ||
} catch (IOException e) { | ||
throw new RuntimeException("Failed to read parquet record", e); | ||
} | ||
return row; | ||
} | ||
|
||
@Override | ||
public void close() throws IOException { | ||
super.close(); | ||
parquetReader.close(); | ||
} | ||
} |
95 changes: 95 additions & 0 deletions
95
...kis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetWriter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,95 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.linkis.storage.resultset; | ||
|
||
import org.apache.linkis.common.io.FsPath; | ||
import org.apache.linkis.common.io.MetaData; | ||
import org.apache.linkis.common.io.Record; | ||
import org.apache.linkis.common.io.resultset.ResultSet; | ||
import org.apache.linkis.storage.domain.Column; | ||
import org.apache.linkis.storage.resultset.table.TableMetaData; | ||
import org.apache.linkis.storage.resultset.table.TableRecord; | ||
|
||
import org.apache.avro.Schema; | ||
import org.apache.avro.SchemaBuilder; | ||
import org.apache.avro.generic.GenericData; | ||
import org.apache.avro.generic.GenericRecord; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.parquet.avro.AvroParquetWriter; | ||
import org.apache.parquet.hadoop.ParquetFileWriter; | ||
import org.apache.parquet.hadoop.ParquetWriter; | ||
import org.apache.parquet.hadoop.metadata.CompressionCodecName; | ||
|
||
import java.io.IOException; | ||
|
||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
public class ParquetResultSetWriter<K extends MetaData, V extends Record> | ||
extends StorageResultSetWriter { | ||
private static final Logger logger = LoggerFactory.getLogger(ParquetResultSetWriter.class); | ||
|
||
private Schema schema = null; | ||
|
||
public ParquetResultSetWriter(ResultSet resultSet, long maxCacheSize, FsPath storePath) { | ||
super(resultSet, maxCacheSize, storePath); | ||
} | ||
|
||
@Override | ||
public void addMetaData(MetaData metaData) throws IOException { | ||
if (!moveToWriteRow) { | ||
rMetaData = metaData; | ||
SchemaBuilder.FieldAssembler<Schema> fieldAssembler = SchemaBuilder.record("linkis").fields(); | ||
TableMetaData tableMetaData = (TableMetaData) rMetaData; | ||
for (Column column : tableMetaData.columns) { | ||
fieldAssembler | ||
.name(column.getColumnName().replaceAll("\\.", "_").replaceAll("[^a-zA-Z0-9_]", "")) | ||
.doc(column.getComment()) | ||
.type(column.getDataType().getTypeName().toLowerCase()) | ||
.noDefault(); | ||
} | ||
schema = fieldAssembler.endRecord(); | ||
moveToWriteRow = true; | ||
} | ||
} | ||
|
||
@Override | ||
public void addRecord(Record record) { | ||
if (moveToWriteRow) { | ||
try { | ||
TableRecord tableRecord = (TableRecord) record; | ||
Object[] row = tableRecord.row; | ||
try (ParquetWriter<GenericRecord> writer = | ||
AvroParquetWriter.<GenericRecord>builder(new Path(storePath.getPath())) | ||
.withSchema(schema) | ||
.withCompressionCodec(CompressionCodecName.SNAPPY) | ||
.withWriteMode(ParquetFileWriter.Mode.OVERWRITE) | ||
.build()) { | ||
|
||
GenericRecord genericRecord = new GenericData.Record(schema); | ||
for (int i = 0; i < row.length; i++) { | ||
genericRecord.put(schema.getFields().get(i).name(), row[i]); | ||
} | ||
writer.write(genericRecord); | ||
} | ||
} catch (Exception e) { | ||
logger.warn("addMetaDataAndRecordString failed", e); | ||
} | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.