yihua commented on code in PR #12866: URL: https://github.com/apache/hudi/pull/12866#discussion_r1978070912
########## hudi-io/src/main/java/org/apache/hudi/io/hfile/HFileWriter.java: ########## @@ -0,0 +1,34 @@ +/* + * 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.hudi.io.hfile; + +import java.io.Closeable; +import java.io.IOException; + +public interface HFileWriter extends Closeable { + // Append a key-value pair into a data block. + void append(byte[] key, byte[] value) throws IOException; Review Comment: Should the `key` use `String` type and let the implementation convert the `String` instance to byte array? I see all callers of `append` convert the key to a byte array which can be simplified. ########## hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java: ########## @@ -102,25 +103,24 @@ public HoodieAvroHFileWriter(String instantTime, StoragePath file, HoodieHFileCo this.taskContextSupplier = taskContextSupplier; this.populateMetaFields = populateMetaFields; - HFileContext context = new HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize()) - .withCompression(hfileConfig.getCompressionAlgorithm()) - .withCellComparator(hfileConfig.getHFileComparator()) + HFileContext context = new HFileContext.Builder() + .withBlockSize(hfileConfig.getBlockSize()) + .withCompressionCodec(hfileConfig.getCompressionCodec()) .build(); conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY, String.valueOf(hfileConfig.shouldPrefetchBlocksOnOpen())); conf.set(HColumnDescriptor.CACHE_DATA_IN_L1, String.valueOf(hfileConfig.shouldCacheDataInL1())); conf.set(DROP_BEHIND_CACHE_COMPACTION_KEY, String.valueOf(hfileConfig.shouldDropBehindCacheCompaction())); - CacheConfig cacheConfig = new CacheConfig(conf); - this.writer = HFile.getWriterFactory(conf, cacheConfig) - .withPath(fs, this.file) - .withFileContext(context) - .create(); - - writer.appendFileInfo(getUTF8Bytes(HoodieAvroHFileReaderImplBase.SCHEMA_KEY), - getUTF8Bytes(schema.toString())); + + FsPermission fsPermission = FsPermission.getFileDefault(); + FSDataOutputStream outputStream = HoodieHFileUtils.create(fs, this.file, fsPermission, true); + this.writer = new HFileWriterImpl(context, outputStream); Review Comment: Could we decouple the HFile writer from Hadoop's `FSDataOutputStream` and other Hadoop classes and move the `HoodieAvroHFileWriter` class to `hudi-common` module? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
