Skip to content

Commit

Permalink
Making the IcebergWriter always use 1 thread for writing to the data …
Browse files Browse the repository at this point in the history
…file (#130)

Co-authored-by: Sundaram Ananthanarayanan <[email protected]>
  • Loading branch information
sundargates and sundargates authored Dec 5, 2021
1 parent 28ce730 commit 667e58c
Show file tree
Hide file tree
Showing 12 changed files with 335 additions and 15 deletions.
16 changes: 15 additions & 1 deletion mantis-connectors/mantis-connector-iceberg/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,24 @@ apply plugin: 'mantis'
ext {
hadoopVersion = '2.7.3'
icebergVersion = '0.9.+'
junitVersion = '5.3.+'
junitVersion = '5.4.+'
mockitoVersion = '2.18.+'
parquetVersion = '1.12.0'
}

dependencies {
configurations {
// we need parquet dependency to be present in testing classpath.
// hence we need to extend from shadow configuration
testImplementation.extendsFrom shadow
all {
// we want a parquet version above 1.12.0 because we need this fix
// https://issues.apache.org/jira/browse/PARQUET-1851
resolutionStrategy {
force "org.apache.parquet:parquet-hadoop:${parquetVersion}"
}
}
}
implementation project(":mantis-runtime")

// We only need the Configuration interface. Users can bring their own hadoop-common version.
Expand All @@ -41,6 +54,7 @@ dependencies {
shadow "org.slf4j:slf4j-log4j12:$slf4jVersion"

testImplementation "org.junit.jupiter:junit-jupiter-api:$junitVersion"
testImplementation "org.junit.jupiter:junit-jupiter-params:$junitVersion"
testRuntimeOnly "org.junit.jupiter:junit-jupiter-engine:$junitVersion"
testImplementation "org.mockito:mockito-core:$mockitoVersion"

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,12 @@
import io.mantisrx.connector.iceberg.sink.committer.config.CommitterConfig;
import io.mantisrx.connector.iceberg.sink.writer.config.WriterConfig;
import io.mantisrx.runtime.parameter.Parameters;
import lombok.RequiredArgsConstructor;

/**
* Convenient base config used by {@link WriterConfig} and {@link CommitterConfig}.
*/
@RequiredArgsConstructor
public class SinkConfig {

private final String catalog;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@

package io.mantisrx.connector.iceberg.sink.writer;

import com.google.common.util.concurrent.ThreadFactoryBuilder;
import io.mantisrx.connector.iceberg.sink.codecs.IcebergCodecs;
import io.mantisrx.connector.iceberg.sink.writer.config.WriterConfig;
import io.mantisrx.connector.iceberg.sink.writer.config.WriterProperties;
Expand All @@ -34,6 +35,8 @@
import io.mantisrx.runtime.parameter.type.IntParameter;
import io.mantisrx.runtime.parameter.type.StringParameter;
import io.mantisrx.runtime.parameter.validator.Validators;
import io.mantisrx.runtime.scheduler.MantisRxSingleThreadScheduler;
import io.mantisrx.shaded.com.google.common.annotations.VisibleForTesting;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
Expand Down Expand Up @@ -75,6 +78,7 @@ public static ScalarToScalar.Config<Record, DataFile> config() {
return new ScalarToScalar.Config<Record, DataFile>()
.description("")
.codec(IcebergCodecs.dataFile())
.serialInput()
.withParameters(parameters());
}

Expand Down Expand Up @@ -119,15 +123,27 @@ public static Transformer newTransformer(Context context) {

LocationProvider locationProvider = context.getServiceLocator().service(LocationProvider.class);
IcebergWriterFactory factory = new DefaultIcebergWriterFactory(config, workerInfo, table, locationProvider);
IcebergWriterPool writerPool = new FixedIcebergWriterPool(
factory,
config.getWriterFlushFrequencyBytes(),
config.getWriterMaximumPoolSize());
IcebergWriterPool writerPool = new FixedIcebergWriterPool(factory, config);
WriterMetrics metrics = new WriterMetrics();
PartitionerFactory partitionerFactory = context.getServiceLocator().service(PartitionerFactory.class);
Partitioner partitioner = partitionerFactory.getPartitioner(table);

return new Transformer(config, metrics, writerPool, partitioner, Schedulers.computation(), Schedulers.io());
return newTransformer(config, metrics, writerPool, partitioner, context.getWorkerInfo());
}

@VisibleForTesting
static Transformer newTransformer(
WriterConfig writerConfig,
WriterMetrics writerMetrics,
IcebergWriterPool writerPool,
Partitioner partitioner,
WorkerInfo workerInfo) {
int workerIdx = workerInfo.getWorkerIndex();
String nameFormat = "IcebergWriter (" + (workerIdx + 1) + ")-%d";
Scheduler executingService = new MantisRxSingleThreadScheduler(
new ThreadFactoryBuilder().setNameFormat(nameFormat).build());
return new Transformer(writerConfig, writerMetrics, writerPool, partitioner,
Schedulers.computation(), executingService);
}

public IcebergWriterStage() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,19 @@ public WriterConfig(Parameters parameters, Configuration hadoopConfig) {
this.hadoopConfig = hadoopConfig;
}

public WriterConfig(String catalog, String database, String table, int writerRowGroupSize,
long writerFlushFrequencyBytes, long writerFlushFrequencyMsec,
String writerFileFormat, int writerMaximumPoolSize,
Configuration hadoopConfig) {
super(catalog, database, table);
this.writerRowGroupSize = writerRowGroupSize;
this.writerFlushFrequencyBytes = writerFlushFrequencyBytes;
this.writerFlushFrequencyMsec = writerFlushFrequencyMsec;
this.writerFileFormat = writerFileFormat;
this.writerMaximumPoolSize = writerMaximumPoolSize;
this.hadoopConfig = hadoopConfig;
}

/**
* Returns an int representing maximum number of rows that should exist in a file.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,10 @@ public class FixedIcebergWriterPool implements IcebergWriterPool {
private final long flushFrequencyBytes;
private final int maximumPoolSize;

public FixedIcebergWriterPool(IcebergWriterFactory factory, WriterConfig writerConfig) {
this(factory, writerConfig.getWriterFlushFrequencyBytes(), writerConfig.getWriterMaximumPoolSize());
}

public FixedIcebergWriterPool(IcebergWriterFactory factory, long flushFrequencyBytes, int maximumPoolSize) {
this.factory = factory;
this.flushFrequencyBytes = flushFrequencyBytes;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,4 +19,4 @@ log4j.rootLogger=INFO, stdout
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
log4j.appender.stdout.Target=System.out
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p %c{1}:%L - %m%n
log4j.appender.stdout.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss} %-5p [%t] %c{1}:%L - %m%n
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* Copyright 2021 Netflix, Inc.
*
* Licensed 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 io.mantisrx.connector.iceberg.sink.writer;

import com.google.common.io.Files;
import java.io.File;
import lombok.Builder;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.io.FileUtils;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.Table;
import org.apache.iceberg.catalog.TableIdentifier;
import org.apache.iceberg.hadoop.HadoopTables;
import org.apache.iceberg.io.LocationProvider;
import org.junit.jupiter.api.extension.AfterEachCallback;
import org.junit.jupiter.api.extension.BeforeAllCallback;
import org.junit.jupiter.api.extension.BeforeEachCallback;
import org.junit.jupiter.api.extension.ExtensionContext;

/**
* Junit Jupiter Extension to create Iceberg Tables for unit testing with a specified schema,
* properties, etc.... and to also clean the files after unit tests.
* <p>
* The way to use the IcebergTableExtension is by adding the following code to your test class. This
* creates the table before the test is executed.
* <pre>
* @RegisterExtension
* static IcebergTableExtension tableExtension =
* IcebergTableExtension.builder()
* .schema(SCHEMA)
* .spec(SPEC)
* .build();
* </pre>
*
* <p> The created table can be obtained by the {@link IcebergTableExtension#getTable()} method.
*/
@Slf4j
@Builder
public class IcebergTableExtension implements BeforeAllCallback, BeforeEachCallback,
AfterEachCallback {

private File rootDir;

@Getter
@Builder.Default
private String catalog = "catalog";

@Getter
@Builder.Default
private String database = "database";

@Getter
@Builder.Default
private String tableName = "table";

@Getter
private Schema schema;
private PartitionSpec spec;

@Getter
private Table table;

@Override
public void beforeAll(ExtensionContext context) throws Exception {
log.info("Before All");
}

@Override
public void beforeEach(ExtensionContext context) throws Exception {
log.info("Before Each");
if (rootDir == null) {
rootDir = Files.createTempDir();
}

final File tableDir = new File(rootDir, getTableIdentifier().toString());
final HadoopTables tables = new HadoopTables();
table = tables.create(schema, spec, tableDir.getPath());
}

@Override
public void afterEach(ExtensionContext context) throws Exception {
FileUtils.deleteDirectory(rootDir);
rootDir = null;
}

public LocationProvider getLocationProvider() {
return table.locationProvider();
}

public TableIdentifier getTableIdentifier() {
return TableIdentifier.of(catalog, database, tableName);
}
}
Loading

0 comments on commit 667e58c

Please sign in to comment.