1
0

[HUDI-2501] Add HoodieData abstraction and refactor compaction actions in hudi-client module (#3741)

This commit is contained in:
Y Ethan Guo
2021-10-22 12:58:51 -07:00
committed by GitHub
parent 1e285dc399
commit 5ed35bff83
41 changed files with 1084 additions and 1019 deletions

View File

@@ -0,0 +1,39 @@
/*
* 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.data;
import java.io.Serializable;
/**
* An abstraction for accumulator on counts.
*/
public abstract class HoodieAccumulator implements Serializable {
/**
* @return the count.
*/
public abstract long value();
/**
* Increments the count based on the input.
*
* @param increment the value to add.
*/
public abstract void add(long increment);
}

View File

@@ -0,0 +1,48 @@
/*
* 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.data;
import java.util.concurrent.atomic.AtomicLong;
/**
* An accumulator on counts based on {@link AtomicLong} implementation.
*/
public class HoodieAtomicLongAccumulator extends HoodieAccumulator {
private final AtomicLong accumulator;
private HoodieAtomicLongAccumulator() {
accumulator = new AtomicLong(0L);
}
public static HoodieAtomicLongAccumulator create() {
return new HoodieAtomicLongAccumulator();
}
@Override
public long value() {
return accumulator.get();
}
@Override
public void add(long increment) {
accumulator.addAndGet(increment);
}
}

View File

@@ -0,0 +1,63 @@
/*
* 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.data;
import org.apache.hudi.common.function.SerializableFunction;
import java.io.Serializable;
import java.util.Iterator;
import java.util.List;
/**
* An abstraction for a data collection of objects in type T to store the reference
* and do transformation.
*
* @param <T> type of object.
*/
public abstract class HoodieData<T> implements Serializable {
/**
* @return the collection of objects.
*/
public abstract Object get();
/**
* @return whether the collection is empty.
*/
public abstract boolean isEmpty();
/**
* @param func serializable map function.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
*/
public abstract <O> HoodieData<O> map(SerializableFunction<T, O> func);
/**
* @param func serializable flatmap function.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
*/
public abstract <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func);
/**
* @return collected results in {@link List<T>}.
*/
public abstract List<T> collectAsList();
}

View File

@@ -0,0 +1,94 @@
/*
* 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.data;
import org.apache.hudi.common.function.SerializableFunction;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
/**
* Holds a {@link List} of objects.
*
* @param <T> type of object.
*/
public class HoodieList<T> extends HoodieData<T> {
private final List<T> listData;
private HoodieList(List<T> listData) {
this.listData = listData;
}
/**
* @param listData a {@link List} of objects in type T.
* @param <T> type of object.
* @return a new instance containing the {@link List<T>} reference.
*/
public static <T> HoodieList<T> of(List<T> listData) {
return new HoodieList<>(listData);
}
/**
* @param hoodieData {@link HoodieList <T>} instance containing the {@link List} of objects.
* @param <T> type of object.
* @return the a {@link List} of objects in type T.
*/
public static <T> List<T> getList(HoodieData<T> hoodieData) {
return ((HoodieList<T>) hoodieData).get();
}
@Override
public List<T> get() {
return listData;
}
@Override
public boolean isEmpty() {
return listData.isEmpty();
}
@Override
public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
return HoodieList.of(listData.stream().parallel()
.map(throwingMapWrapper(func)).collect(Collectors.toList()));
}
@Override
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
Function<T, Iterator<O>> throwableFunc = throwingMapWrapper(func);
return HoodieList.of(listData.stream().flatMap(e -> {
List<O> result = new ArrayList<>();
Iterator<O> iterator = throwableFunc.apply(e);
iterator.forEachRemaining(result::add);
return result.stream();
}).collect(Collectors.toList()));
}
@Override
public List<T> collectAsList() {
return listData;
}
}

View File

@@ -19,6 +19,8 @@
package org.apache.hudi.common.engine;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.data.HoodieAccumulator;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.function.SerializableBiFunction;
import org.apache.hudi.common.function.SerializableConsumer;
import org.apache.hudi.common.function.SerializableFunction;
@@ -59,6 +61,12 @@ public abstract class HoodieEngineContext {
return taskContextSupplier;
}
public abstract HoodieAccumulator newAccumulator();
public abstract <T> HoodieData<T> emptyHoodieData();
public abstract <T> HoodieData<T> parallelize(List<T> data);
public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism);
public abstract <I, K, V> List<V> mapToPairAndReduceByKey(

View File

@@ -21,6 +21,10 @@ package org.apache.hudi.common.engine;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.data.HoodieAccumulator;
import org.apache.hudi.common.data.HoodieAtomicLongAccumulator;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.function.SerializableBiFunction;
import org.apache.hudi.common.function.SerializableConsumer;
import org.apache.hudi.common.function.SerializableFunction;
@@ -31,6 +35,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.common.util.collection.Pair;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -59,6 +64,21 @@ public final class HoodieLocalEngineContext extends HoodieEngineContext {
super(new SerializableConfiguration(conf), taskContextSupplier);
}
@Override
public HoodieAccumulator newAccumulator() {
return HoodieAtomicLongAccumulator.create();
}
@Override
public <T> HoodieData<T> emptyHoodieData() {
return HoodieList.of(Collections.emptyList());
}
@Override
public <T> HoodieData<T> parallelize(List<T> data) {
return HoodieList.of(data);
}
@Override
public <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism) {
return data.stream().parallel().map(throwingMapWrapper(func)).collect(toList());