diff --git a/linkis-commons/linkis-storage/pom.xml b/linkis-commons/linkis-storage/pom.xml index def795ebd8..c9c610bcbd 100644 --- a/linkis-commons/linkis-storage/pom.xml +++ b/linkis-commons/linkis-storage/pom.xml @@ -99,6 +99,36 @@ aws-java-sdk-s3 1.12.261 + + + org.apache.parquet + parquet-avro + 1.12.0 + + + org.apache.hadoop + hadoop-mapreduce-client-core + ${hadoop.version} + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + + ch.qos.reload4j + reload4j + + + org.slf4j + slf4j-reload4j + + + diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java index 74950c15fe..916f9ff469 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/conf/LinkisStorageConf.java @@ -25,6 +25,12 @@ public class LinkisStorageConf { private static final Object CONF_LOCK = new Object(); + public static final String DOLPHIN = "dolphin"; + + public static final String PARQUET = "parquet"; + + public static final String PARQUET_FILE_SUFFIX = ".parquet"; + public static final String HDFS_FILE_SYSTEM_REST_ERRS = CommonVars.apply( "wds.linkis.hdfs.rest.errs", @@ -34,12 +40,17 @@ public class LinkisStorageConf { public static final String ROW_BYTE_MAX_LEN_STR = CommonVars.apply("wds.linkis.resultset.row.max.str", "2m").getValue(); + public static final String ENGINE_RESULT_TYPE = + CommonVars.apply("wds.linkis.engine.resultSet.type", DOLPHIN, "Result type").getValue(); + public static final long ROW_BYTE_MAX_LEN = ByteTimeUtils.byteStringAsBytes(ROW_BYTE_MAX_LEN_STR); public static final String FILE_TYPE = CommonVars.apply( "wds.linkis.storage.file.type", - "dolphin,sql,scala,py,hql,python,out,log,text,txt,sh,jdbc,ngql,psql,fql,tsql") + "dolphin,sql,scala,py,hql,python,out,log,text,txt,sh,jdbc,ngql,psql,fql,tsql" + + "," + + PARQUET) .getValue(); private static volatile String[] fileTypeArr = null; diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java index db78afac29..9ac4c02cc7 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/DefaultResultSetFactory.java @@ -23,6 +23,7 @@ import org.apache.linkis.common.io.Record; import org.apache.linkis.common.io.resultset.ResultSet; import org.apache.linkis.storage.FSFactory; +import org.apache.linkis.storage.conf.LinkisStorageConf; import org.apache.linkis.storage.domain.Dolphin; import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary; import org.apache.linkis.storage.exception.StorageWarnException; @@ -134,15 +135,22 @@ public String[] getResultSetType() { @Override public ResultSet getResultSetByPath(FsPath fsPath, Fs fs) { + ResultSet resultSet = null; try (InputStream inputStream = fs.read(fsPath)) { - String resultSetType = Dolphin.getType(inputStream); - if (StringUtils.isEmpty(resultSetType)) { - throw new StorageWarnException( - THE_FILE_IS_EMPTY.getErrorCode(), - MessageFormat.format(THE_FILE_IS_EMPTY.getErrorDesc(), fsPath.getPath())); + String engineResultType = LinkisStorageConf.ENGINE_RESULT_TYPE; + if (engineResultType.equals(LinkisStorageConf.DOLPHIN)) { + String resultSetType = Dolphin.getType(inputStream); + if (StringUtils.isEmpty(resultSetType)) { + throw new StorageWarnException( + THE_FILE_IS_EMPTY.getErrorCode(), + MessageFormat.format(THE_FILE_IS_EMPTY.getErrorDesc(), fsPath.getPath())); + } + // Utils.tryQuietly(fs::close); + resultSet = getResultSetByType(resultSetType); + } else if (engineResultType.equals(LinkisStorageConf.PARQUET)) { + resultSet = getResultSetByType(ResultSetFactory.TABLE_TYPE); } - // Utils.tryQuietly(fs::close); - return getResultSetByType(resultSetType); + return resultSet; } catch (IOException e) { throw new RuntimeException(e); } diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetReader.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetReader.java new file mode 100644 index 0000000000..07c6c4cd7a --- /dev/null +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetReader.java @@ -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 + extends StorageResultSetReader { + + private static final Logger logger = LoggerFactory.getLogger(ParquetResultSetReader.class); + + private FsPath fsPath; + + private ParquetReader parquetReader; + + private GenericRecord record; + + public ParquetResultSetReader(ResultSet resultSet, InputStream inputStream, FsPath fsPath) + throws IOException { + super(resultSet, inputStream); + this.fsPath = fsPath; + this.parquetReader = + AvroParquetReader.builder(new Path(fsPath.getPath())).build(); + this.record = parquetReader.read(); + } + + @Override + public MetaData getMetaData() { + if (metaData == null) { + try { + List fields = record.getSchema().getFields(); + List 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 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(); + } +} diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetWriter.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetWriter.java new file mode 100644 index 0000000000..31dd6c7a5c --- /dev/null +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ParquetResultSetWriter.java @@ -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 + 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 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 writer = + AvroParquetWriter.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); + } + } + } +} diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java index 5e56b099d7..03434abdbe 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetReaderFactory.java @@ -24,6 +24,7 @@ import org.apache.linkis.common.io.resultset.ResultSet; import org.apache.linkis.common.io.resultset.ResultSetReader; import org.apache.linkis.storage.FSFactory; +import org.apache.linkis.storage.conf.LinkisStorageConf; import org.apache.linkis.storage.errorcode.LinkisStorageErrorCodeSummary; import org.apache.linkis.storage.exception.StorageWarnException; import org.apache.linkis.storage.resultset.table.TableMetaData; @@ -40,8 +41,19 @@ public class ResultSetReaderFactory { private static final Logger logger = LoggerFactory.getLogger(ResultSetReaderFactory.class); public static ResultSetReader getResultSetReader( - ResultSet resultSet, InputStream inputStream) { - return new StorageResultSetReader<>(resultSet, inputStream); + ResultSet resultSet, InputStream inputStream, FsPath fsPath) { + String engineResultType = LinkisStorageConf.ENGINE_RESULT_TYPE; + StorageResultSetReader resultSetReader = null; + if (engineResultType.equals(LinkisStorageConf.DOLPHIN)) { + resultSetReader = new StorageResultSetReader<>(resultSet, inputStream); + } else if (engineResultType.equals(LinkisStorageConf.PARQUET)) { + try { + resultSetReader = new ParquetResultSetReader<>(resultSet, inputStream, fsPath); + } catch (IOException e) { + throw new RuntimeException("Failed to read parquet", e); + } + } + return resultSetReader; } public static ResultSetReader getResultSetReader( @@ -61,7 +73,7 @@ public static ResultSetReader getResultSetReader(String res) throws IOException Fs fs = FSFactory.getFs(resPath); fs.init(null); ResultSetReader reader = - ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath)); + ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath), resPath); if (reader instanceof StorageResultSetReader) { ((StorageResultSetReader) reader).setFs(fs); } @@ -96,7 +108,7 @@ public static ResultSetReader getTableResultReader(String res) { InputStream read = fs.read(resPath); return ResultSetReaderFactory.getResultSetReader( - (TableResultSet) resultSet, read); + (TableResultSet) resultSet, read, resPath); } catch (IOException e) { throw new StorageWarnException( LinkisStorageErrorCodeSummary.TABLE_ARE_NOT_SUPPORTED.getErrorCode(), diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java index d70319c9bd..3e50692186 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/ResultSetWriterFactory.java @@ -22,6 +22,7 @@ import org.apache.linkis.common.io.Record; import org.apache.linkis.common.io.resultset.ResultSet; import org.apache.linkis.common.io.resultset.ResultSetReader; +import org.apache.linkis.storage.conf.LinkisStorageConf; import java.io.IOException; import java.util.ArrayList; @@ -36,14 +37,26 @@ public class ResultSetWriterFactory { public static org.apache.linkis.common.io.resultset.ResultSetWriter getResultSetWriter( ResultSet resultSet, long maxCacheSize, FsPath storePath) { - return new StorageResultSetWriter<>(resultSet, maxCacheSize, storePath); + String engineResultType = LinkisStorageConf.ENGINE_RESULT_TYPE; + StorageResultSetWriter writer = null; + if (engineResultType.equals(LinkisStorageConf.DOLPHIN)) { + writer = new StorageResultSetWriter<>(resultSet, maxCacheSize, storePath); + } else if (engineResultType.equals(LinkisStorageConf.PARQUET)) { + writer = new ParquetResultSetWriter<>(resultSet, maxCacheSize, storePath); + } + return writer; } public static org.apache.linkis.common.io.resultset.ResultSetWriter getResultSetWriter( ResultSet resultSet, long maxCacheSize, FsPath storePath, String proxyUser) { - StorageResultSetWriter writer = - new StorageResultSetWriter<>(resultSet, maxCacheSize, storePath); + String engineResultType = LinkisStorageConf.ENGINE_RESULT_TYPE; + StorageResultSetWriter writer = null; + if (engineResultType.equals(LinkisStorageConf.DOLPHIN)) { + writer = new StorageResultSetWriter<>(resultSet, maxCacheSize, storePath); + } else if (engineResultType.equals(LinkisStorageConf.PARQUET)) { + writer = new ParquetResultSetWriter<>(resultSet, maxCacheSize, storePath); + } writer.setProxyUser(proxyUser); return writer; } diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java index c83661de2e..9ec8f27cb3 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSet.java @@ -21,6 +21,7 @@ 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.conf.LinkisStorageConf; import org.apache.linkis.storage.domain.Dolphin; import org.apache.linkis.storage.utils.StorageConfiguration; @@ -49,10 +50,17 @@ public String charset() { @Override public FsPath getResultSetPath(FsPath parentDir, String fileName) { + String engineResultType = LinkisStorageConf.ENGINE_RESULT_TYPE; + String fileSuffix = null; + if (engineResultType.equals(LinkisStorageConf.DOLPHIN)) { + fileSuffix = Dolphin.DOLPHIN_FILE_SUFFIX; + } else if (engineResultType.equals(LinkisStorageConf.PARQUET)) { + fileSuffix = LinkisStorageConf.PARQUET_FILE_SUFFIX; + } final String path = parentDir.getPath().endsWith("/") - ? parentDir.getUriString() + fileName + Dolphin.DOLPHIN_FILE_SUFFIX - : parentDir.getUriString() + "/" + fileName + Dolphin.DOLPHIN_FILE_SUFFIX; + ? parentDir.getUriString() + fileName + fileSuffix + : parentDir.getUriString() + "/" + fileName + fileSuffix; logger.info("Get result set path: {}", path); return new FsPath(path); } diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java index c0222cc848..595ebda983 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetReader.java @@ -43,10 +43,10 @@ public class StorageResultSetReader private static final Logger logger = LoggerFactory.getLogger(StorageResultSetReader.class); private final ResultSet resultSet; - private final InputStream inputStream; - private final ResultDeserializer deserializer; - private K metaData; - private Record row; + public final InputStream inputStream; + public final ResultDeserializer deserializer; + public K metaData; + public Record row; private int colCount = 0; private int rowCount = 0; private Fs fs; diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java index ea513664bd..bdea8ee4e7 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/resultset/StorageResultSetWriter.java @@ -17,15 +17,19 @@ package org.apache.linkis.storage.resultset; -import org.apache.linkis.common.io.*; -import org.apache.linkis.common.io.resultset.*; +import org.apache.linkis.common.io.Fs; +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.ResultSerializer; +import org.apache.linkis.common.io.resultset.ResultSet; import org.apache.linkis.common.io.resultset.ResultSetWriter; -import org.apache.linkis.common.utils.*; -import org.apache.linkis.storage.*; -import org.apache.linkis.storage.conf.*; -import org.apache.linkis.storage.domain.*; +import org.apache.linkis.storage.FSFactory; +import org.apache.linkis.storage.conf.LinkisStorageConf; +import org.apache.linkis.storage.domain.Dolphin; import org.apache.linkis.storage.exception.StorageErrorException; -import org.apache.linkis.storage.utils.*; +import org.apache.linkis.storage.utils.FileSystemUtils; +import org.apache.linkis.storage.utils.StorageUtils; import org.apache.commons.io.IOUtils; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; @@ -46,15 +50,15 @@ public class StorageResultSetWriter private final ResultSet resultSet; private final long maxCacheSize; - private final FsPath storePath; + public final FsPath storePath; private final ResultSerializer serializer; - private boolean moveToWriteRow = false; + public boolean moveToWriteRow = false; private OutputStream outputStream = null; private int rowCount = 0; private final List buffer = new ArrayList(); private Fs fs = null; - private MetaData rMetaData = null; + public MetaData rMetaData = null; private String proxyUser = StorageUtils.getJvmUser(); private boolean fileCreated = false; private boolean closed = false; diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java index 0ed650186d..9a71e64420 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/source/FileSource.java @@ -73,7 +73,12 @@ public interface FileSource extends Closeable { (path, suffix) -> path.endsWith("." + suffix); static boolean isResultSet(String path) { - return suffixPredicate.apply(path, fileType[0]); + String engineResultType = LinkisStorageConf.ENGINE_RESULT_TYPE; + String type = fileType[0]; + if (engineResultType.equals(LinkisStorageConf.PARQUET)) { + type = LinkisStorageConf.PARQUET; + } + return suffixPredicate.apply(path, type); } static boolean isResultSet(FsPath fsPath) { @@ -137,7 +142,8 @@ static FileSource create(FsPath fsPath, InputStream is) { static FileSplit createResultSetFileSplit(FsPath fsPath, InputStream is) { logger.info("try create result set file split with path:{}", fsPath.getPath()); ResultSet resultset = ResultSetFactory.getInstance().getResultSetByPath(fsPath); - ResultSetReader resultsetReader = ResultSetReaderFactory.getResultSetReader(resultset, is); + ResultSetReader resultsetReader = + ResultSetReaderFactory.getResultSetReader(resultset, is, fsPath); return new FileSplit(resultsetReader, resultset.resultSetType()); } @@ -145,7 +151,8 @@ static FileSplit createResultSetFileSplit(FsPath fsPath, Fs fs) { ResultSet resultset = ResultSetFactory.getInstance().getResultSetByPath(fsPath, fs); ResultSetReader resultsetReader = null; try { - resultsetReader = ResultSetReaderFactory.getResultSetReader(resultset, fs.read(fsPath)); + resultsetReader = + ResultSetReaderFactory.getResultSetReader(resultset, fs.read(fsPath), fsPath); } catch (IOException e) { logger.warn("FileSource createResultSetFileSplit failed", e); } diff --git a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java index e1dee151ca..491c3d7af4 100644 --- a/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java +++ b/linkis-commons/linkis-storage/src/main/java/org/apache/linkis/storage/utils/StorageHelper.java @@ -80,7 +80,8 @@ public static void getTableResLines(String[] args) { resultSetFactory.getResultSetByType(ResultSetFactory.TABLE_TYPE); Fs fs = FSFactory.getFs(resPath); fs.init(null); - resultSetReader = ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath)); + resultSetReader = + ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath), resPath); TableMetaData metaData = (TableMetaData) resultSetReader.getMetaData(); Arrays.stream(metaData.getColumns()).forEach(column -> logger.info(column.toString())); int num = 0; @@ -116,7 +117,7 @@ public static void getTableRes(String[] args) { fs.init(null); ResultSetReader reader = - ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath)); + ResultSetReaderFactory.getResultSetReader(resultSet, fs.read(resPath), resPath); MetaData rmetaData = reader.getMetaData(); Arrays.stream(((TableMetaData) rmetaData).getColumns()) .forEach(column -> logger.info(column.toString())); diff --git a/linkis-public-enhancements/linkis-pes-publicservice/src/main/java/org/apache/linkis/filesystem/restful/api/FsRestfulApi.java b/linkis-public-enhancements/linkis-pes-publicservice/src/main/java/org/apache/linkis/filesystem/restful/api/FsRestfulApi.java index 189ab711fd..2af3caaaba 100644 --- a/linkis-public-enhancements/linkis-pes-publicservice/src/main/java/org/apache/linkis/filesystem/restful/api/FsRestfulApi.java +++ b/linkis-public-enhancements/linkis-pes-publicservice/src/main/java/org/apache/linkis/filesystem/restful/api/FsRestfulApi.java @@ -404,6 +404,10 @@ public Message getDirFileTrees( FsPathListWithError fsPathListWithError = fileSystem.listPathWithError(fsPath); if (fsPathListWithError != null) { for (FsPath children : fsPathListWithError.getFsPaths()) { + // 兼容parquet,跳过parquet.crc文件 + if (children.getPath().endsWith(".parquet.crc")) { + continue; + } DirFileTree dirFileTreeChildren = new DirFileTree(); dirFileTreeChildren.setName(new File(children.getPath()).getName()); dirFileTreeChildren.setPath(children.getSchemaPath());