1
0

[HUDI-1510] Move HoodieEngineContext and its dependencies to hudi-common (#2410)

This commit is contained in:
Udit Mehrotra
2021-01-07 11:34:06 -08:00
committed by GitHub
parent 5ff8e88d58
commit 17df517b81
154 changed files with 190 additions and 180 deletions

View File

@@ -0,0 +1,33 @@
/*
* 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.common.engine;
/**
* Properties specific to each engine, that can be set/obtained from.
*/
public enum EngineProperty {
// hostname to bind embedded timeline server to
EMBEDDED_SERVER_HOST,
// Pool/queue to use to run compaction.
COMPACTION_POOL_NAME,
// Amount of total memory available to each engine executor
TOTAL_MEMORY_AVAILABLE,
// Fraction of that memory, that is already in use by the engine
MEMORY_FRACTION_IN_USE,
}

View File

@@ -0,0 +1,26 @@
/*
* 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.common.engine;
/**
* Hoodie data processing engine. support only Apache Spark and Apache Flink for now.
*/
public enum EngineType {
SPARK, FLINK
}

View File

@@ -0,0 +1,71 @@
/*
* 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.common.engine;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.function.SerializableConsumer;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.Option;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
/**
* Base class contains the context information needed by the engine at runtime. It will be extended by different
* engine implementation if needed.
*/
public abstract class HoodieEngineContext {
/**
* A wrapped hadoop configuration which can be serialized.
*/
private SerializableConfiguration hadoopConf;
protected TaskContextSupplier taskContextSupplier;
public HoodieEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) {
this.hadoopConf = hadoopConf;
this.taskContextSupplier = taskContextSupplier;
}
public SerializableConfiguration getHadoopConf() {
return hadoopConf;
}
public TaskContextSupplier getTaskContextSupplier() {
return taskContextSupplier;
}
public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism);
public abstract <I, O> List<O> flatMap(List<I> data, SerializableFunction<I, Stream<O>> func, int parallelism);
public abstract <I> void foreach(List<I> data, SerializableConsumer<I> consumer, int parallelism);
public abstract <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism);
public abstract void setProperty(EngineProperty key, String value);
public abstract Option<String> getProperty(EngineProperty key);
public abstract void setJobStatus(String activeModule, String activityDescription);
}

View File

@@ -0,0 +1,38 @@
/*
* 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.common.engine;
import org.apache.hudi.common.util.Option;
import java.io.Serializable;
import java.util.function.Supplier;
/**
* Base task context supplier.
*/
public abstract class TaskContextSupplier implements Serializable {
public abstract Supplier<Integer> getPartitionIdSupplier();
public abstract Supplier<Integer> getStageIdSupplier();
public abstract Supplier<Long> getAttemptIdSupplier();
public abstract Option<String> getProperty(EngineProperty prop);
}

View File

@@ -0,0 +1,73 @@
/*
* 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.common.function;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Stream;
/**
* Function wrapper util class, which catches the exception thrown by input function and return a similar function
* with no exception thrown.
*/
public class FunctionWrapper {
public static <I, O> Function<I, O> throwingMapWrapper(SerializableFunction<I, O> throwingMapFunction) {
return v1 -> {
try {
return throwingMapFunction.apply(v1);
} catch (Exception e) {
throw new HoodieException("Error occurs when executing map", e);
}
};
}
public static <I, O> Function<I, Stream<O>> throwingFlatMapWrapper(SerializableFunction<I, Stream<O>> throwingFlatMapFunction) {
return v1 -> {
try {
return throwingFlatMapFunction.apply(v1);
} catch (Exception e) {
throw new HoodieException("Error occurs when executing flatMap", e);
}
};
}
public static <I> Consumer<I> throwingForeachWrapper(SerializableConsumer<I> throwingConsumer) {
return v1 -> {
try {
throwingConsumer.accept(v1);
} catch (Exception e) {
throw new HoodieException("Error occurs when executing foreach", e);
}
};
}
public static <I, K, V> Function<I, Pair<K, V>> throwingMapToPairWrapper(SerializablePairFunction<I, K, V> throwingPairFunction) {
return v1 -> {
try {
return throwingPairFunction.call(v1);
} catch (Exception e) {
throw new HoodieException("Error occurs when executing mapToPair", e);
}
};
}
}

View File

@@ -0,0 +1,31 @@
/*
* 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.common.function;
import java.io.Serializable;
/**
* A wrapped {@link java.util.function.Consumer} which can be serialized.
*
* @param <I> input type
*/
@FunctionalInterface
public interface SerializableConsumer<I> extends Serializable {
void accept(I t) throws Exception;
}

View File

@@ -0,0 +1,32 @@
/*
* 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.common.function;
import java.io.Serializable;
/**
* A wrapped {@link java.util.function.Function} which can be serialized.
*
* @param <I> input data type
* @param <O> output data type
*/
@FunctionalInterface
public interface SerializableFunction<I, O> extends Serializable {
O apply(I v1) throws Exception;
}

View File

@@ -0,0 +1,31 @@
/*
* 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.common.function;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
/**
* A function that returns key-value pairs (Tuple2&lt;K, V&gt;).
*/
@FunctionalInterface
public interface SerializablePairFunction<I, K, V> extends Serializable {
Pair<K, V> call(I t) throws Exception;
}