1
0

[HUDI-1468] Support custom clustering strategies and preserve commit metadata as part of clustering (#3419)

Co-authored-by: Satish Kotha <satishkotha@uber.com>
This commit is contained in:
Sagar Sumit
2021-08-07 08:23:08 +05:30
committed by GitHub
parent 9ce548edb1
commit 70b6bd485f
34 changed files with 1150 additions and 343 deletions

View File

@@ -45,6 +45,11 @@
"name":"version",
"type":["int", "null"],
"default": 1
},
{
"name":"preserveHoodieMetadata",
"type":["null", "boolean"],
"default": null
}
]
}

View File

@@ -0,0 +1,89 @@
/*
* 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.model;
import org.apache.hudi.avro.model.HoodieClusteringGroup;
import java.io.Serializable;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
/**
* Encapsulates all the needed information about a clustering group. This is needed because spark serialization
* does not work with avro objects.
*/
public class ClusteringGroupInfo implements Serializable {
private List<ClusteringOperation> operations;
private int numOutputGroups;
public static ClusteringGroupInfo create(HoodieClusteringGroup clusteringGroup) {
List<ClusteringOperation> operations = clusteringGroup.getSlices().stream()
.map(ClusteringOperation::create).collect(Collectors.toList());
return new ClusteringGroupInfo(operations, clusteringGroup.getNumOutputFileGroups());
}
// Only for serialization/de-serialization
@Deprecated
public ClusteringGroupInfo() {}
private ClusteringGroupInfo(final List<ClusteringOperation> operations, final int numOutputGroups) {
this.operations = operations;
this.numOutputGroups = numOutputGroups;
}
public List<ClusteringOperation> getOperations() {
return this.operations;
}
public void setOperations(final List<ClusteringOperation> operations) {
this.operations = operations;
}
public int getNumOutputGroups() {
return this.numOutputGroups;
}
public void setNumOutputGroups(final int numOutputGroups) {
this.numOutputGroups = numOutputGroups;
}
@Override
public boolean equals(final Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final ClusteringGroupInfo that = (ClusteringGroupInfo) o;
return Objects.equals(getFilePathsInGroup(), that.getFilePathsInGroup());
}
@Override
public int hashCode() {
return Objects.hash(getFilePathsInGroup());
}
private String getFilePathsInGroup() {
return getOperations().stream().map(op -> op.getDataFilePath()).collect(Collectors.joining(","));
}
}

View File

@@ -0,0 +1,53 @@
/*
* 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.model;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hudi.common.util.Option;
import java.io.IOException;
/**
* Default payload used for rewrite use cases where we dont change schema. We dont need to serialize/deserialize avro record in payload.
*/
public class RewriteAvroPayload implements HoodieRecordPayload<RewriteAvroPayload> {
private GenericRecord record;
public RewriteAvroPayload(GenericRecord record) {
this.record = record;
}
@Override
public RewriteAvroPayload preCombine(RewriteAvroPayload another) {
throw new UnsupportedOperationException("precombine is not expected for rewrite payload");
}
@Override
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException {
return getInsertValue(schema);
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Option.of(record);
}
}

View File

@@ -35,16 +35,16 @@ import java.util.Iterator;
/**
* Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
*/
public class HoodieFileSliceReader implements Iterator<HoodieRecord<? extends HoodieRecordPayload>> {
private Iterator<HoodieRecord<? extends HoodieRecordPayload>> recordsIterator;
public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
private Iterator<HoodieRecord<T>> recordsIterator;
public static <R extends IndexedRecord, T extends HoodieRecordPayload> HoodieFileSliceReader getFileSliceReader(
public static <R extends IndexedRecord, T> HoodieFileSliceReader getFileSliceReader(
HoodieFileReader<R> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
Option<Pair<String,String>> simpleKeyGenFieldsOpt) throws IOException {
Iterator<R> baseIterator = baseFileReader.getRecordIterator(schema);
while (baseIterator.hasNext()) {
GenericRecord record = (GenericRecord) baseIterator.next();
HoodieRecord<T> hoodieRecord = simpleKeyGenFieldsOpt.isPresent()
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = simpleKeyGenFieldsOpt.isPresent()
? SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass, simpleKeyGenFieldsOpt.get())
: SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass);
scanner.processNextRecord(hoodieRecord);
@@ -52,7 +52,7 @@ public class HoodieFileSliceReader implements Iterator<HoodieRecord<? extends Ho
return new HoodieFileSliceReader(scanner.iterator());
}
private HoodieFileSliceReader(Iterator<HoodieRecord<? extends HoodieRecordPayload>> recordsItr) {
private HoodieFileSliceReader(Iterator<HoodieRecord<T>> recordsItr) {
this.recordsIterator = recordsItr;
}
@@ -62,7 +62,7 @@ public class HoodieFileSliceReader implements Iterator<HoodieRecord<? extends Ho
}
@Override
public HoodieRecord<? extends HoodieRecordPayload> next() {
public HoodieRecord<T> next() {
return recordsIterator.next();
}
}