1
0

[HUDI-328] Adding delete api to HoodieWriteClient (#1004)

[HUDI-328]  Adding delete api to HoodieWriteClient and Spark DataSource
This commit is contained in:
Sivabalan Narayanan
2019-11-22 15:05:25 -08:00
committed by Balaji Varadarajan
parent 7bc08cbfdc
commit c3355109b1
18 changed files with 818 additions and 172 deletions

View File

@@ -92,9 +92,9 @@ public class DataSourceUtils {
/**
* Create a key generator class via reflection, passing in any configs needed.
*
* If the class name of key generator is configured through the properties file, i.e., {@code
* props}, use the corresponding key generator class; otherwise, use the default key generator class specified in
* {@code DataSourceWriteOptions}.
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the
* corresponding key generator class; otherwise, use the default key generator class specified in {@code
* DataSourceWriteOptions}.
*/
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
String keyGeneratorClass = props.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
@@ -124,7 +124,7 @@ public class DataSourceUtils {
throws IOException {
try {
return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
new Class<?>[] {GenericRecord.class, Comparable.class}, record, orderingVal);
new Class<?>[]{GenericRecord.class, Comparable.class}, record, orderingVal);
} catch (Throwable e) {
throw new IOException("Could not create payload for class: " + payloadClass, e);
}
@@ -172,6 +172,11 @@ public class DataSourceUtils {
}
}
public static JavaRDD<WriteStatus> doDeleteOperation(HoodieWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
String commitTime) {
return client.delete(hoodieKeys, commitTime);
}
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey,
String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);

View File

@@ -1,48 +0,0 @@
/*
* 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;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
/**
* Empty payload used for deletions
*/
public class EmptyHoodieRecordPayload implements HoodieRecordPayload<EmptyHoodieRecordPayload> {
public EmptyHoodieRecordPayload(GenericRecord record, Comparable orderingVal) {}
@Override
public EmptyHoodieRecordPayload preCombine(EmptyHoodieRecordPayload another) {
return another;
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) {
return Option.empty();
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) {
return Option.empty();
}
}