Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,13 +18,17 @@

package org.apache.hudi.io.storage;

import static org.apache.hudi.common.model.HoodieFileFormat.HOODIE_LOG;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;

import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.parquet.avro.AvroSchemaConverter;
Expand All @@ -34,9 +38,12 @@ public class HoodieStorageWriterFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
String commitTime, Path path, HoodieTable<T> hoodieTable,
HoodieWriteConfig config, Schema schema) throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
final String name = path.getName();
final String extension = FSUtils.isLogFile(path) ? HOODIE_LOG.getFileExtension() : FSUtils.getFileExtension(name);
if (PARQUET.getFileExtension().equals(extension)) {
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
}
throw new UnsupportedOperationException(extension + " format not supported yet.");
}

private static <T extends HoodieRecordPayload,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,62 @@
/*
* 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.storage;

import static org.junit.Assert.fail;

import java.io.IOException;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.TestHoodieClientBase;
import org.apache.hudi.common.HoodieTestDataGenerator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.junit.Assert;
import org.junit.Test;

/**
* Tests for {@link HoodieStorageWriterFactory}.
*/
public class TestHoodieStorageWriterFactory extends TestHoodieClientBase {

@Test
public void testGetStorageWriter() throws IOException {
// parquet file format.
final String commitTime = "100";
final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet");
final HoodieWriteConfig cfg = getConfig();
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
HoodieStorageWriter<IndexedRecord> parquetWriter =
HoodieStorageWriterFactory.getStorageWriter(
commitTime, parquetPath, table, cfg, HoodieTestDataGenerator.avroSchema);
Assert.assertTrue(parquetWriter instanceof HoodieParquetWriter);

// other file format exception.
final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
try {
HoodieStorageWriter<IndexedRecord> logWriter =
HoodieStorageWriterFactory.getStorageWriter(
commitTime, logPath, table, cfg, HoodieTestDataGenerator.avroSchema);
fail("should fail since log storage writer is not supported yet.");
} catch (Exception e) {
Assert.assertTrue(e instanceof UnsupportedOperationException);
Assert.assertTrue(e.getMessage().contains("format not supported yet."));
}
}
}