[HUDI-1075] Implement simple clustering strategies to create ClusteringPlan and to run the plan
This commit is contained in:
@@ -40,6 +40,11 @@
|
||||
}],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"name":"numOutputFileGroups",
|
||||
"type":["int", "null"],
|
||||
"default": 1
|
||||
},
|
||||
{
|
||||
"name":"version",
|
||||
"type":["int", "null"],
|
||||
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.ObjectOutputStream;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* A wrapped Schema which can be serialized.
|
||||
*/
|
||||
public class SerializableSchema implements Serializable {
|
||||
|
||||
private transient Schema schema;
|
||||
|
||||
public SerializableSchema() {
|
||||
}
|
||||
|
||||
public SerializableSchema(Schema schema) {
|
||||
this.schema = newCopy(schema);
|
||||
}
|
||||
|
||||
public SerializableSchema(SerializableSchema serializableSchema) {
|
||||
this(serializableSchema.schema);
|
||||
}
|
||||
|
||||
public static Schema newCopy(Schema schemaObject) {
|
||||
return new Schema.Parser().parse(schemaObject.toString());
|
||||
}
|
||||
|
||||
public Schema get() {
|
||||
return schema;
|
||||
}
|
||||
|
||||
private void writeObject(ObjectOutputStream out) throws IOException {
|
||||
out.defaultWriteObject();
|
||||
writeObjectTo(out);
|
||||
}
|
||||
|
||||
private void readObject(ObjectInputStream in) throws IOException {
|
||||
readObjectFrom(in);
|
||||
}
|
||||
|
||||
// create a public write method for unit test
|
||||
public void writeObjectTo(ObjectOutputStream out) throws IOException {
|
||||
out.writeUTF(schema.toString());
|
||||
}
|
||||
|
||||
// create a public read method for unit test
|
||||
public void readObjectFrom(ObjectInputStream in) throws IOException {
|
||||
schema = new Schema.Parser().parse(in.readUTF());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return schema.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,129 @@
|
||||
/*
|
||||
* 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.HoodieSliceInfo;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Encapsulates all the needed information about a clustering file slice. This is needed because spark serialization
|
||||
* does not work with avro objects.
|
||||
*/
|
||||
public class ClusteringOperation implements Serializable {
|
||||
|
||||
private String dataFilePath;
|
||||
private List<String> deltaFilePaths;
|
||||
private String fileId;
|
||||
private String partitionPath;
|
||||
private String bootstrapFilePath;
|
||||
private int version;
|
||||
|
||||
public static ClusteringOperation create(HoodieSliceInfo sliceInfo) {
|
||||
return new ClusteringOperation(sliceInfo.getDataFilePath(), new ArrayList<>(sliceInfo.getDeltaFilePaths()), sliceInfo.getFileId(),
|
||||
sliceInfo.getPartitionPath(), sliceInfo.getBootstrapFilePath(), sliceInfo.getVersion());
|
||||
}
|
||||
|
||||
// Only for serialization/de-serialization
|
||||
@Deprecated
|
||||
public ClusteringOperation() {}
|
||||
|
||||
private ClusteringOperation(final String dataFilePath, final List<String> deltaFilePaths, final String fileId,
|
||||
final String partitionPath, final String bootstrapFilePath, final int version) {
|
||||
this.dataFilePath = dataFilePath;
|
||||
this.deltaFilePaths = deltaFilePaths;
|
||||
this.fileId = fileId;
|
||||
this.partitionPath = partitionPath;
|
||||
this.bootstrapFilePath = bootstrapFilePath;
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
public String getDataFilePath() {
|
||||
return this.dataFilePath;
|
||||
}
|
||||
|
||||
public void setDataFilePath(final String dataFilePath) {
|
||||
this.dataFilePath = dataFilePath;
|
||||
}
|
||||
|
||||
public List<String> getDeltaFilePaths() {
|
||||
return this.deltaFilePaths;
|
||||
}
|
||||
|
||||
public void setDeltaFilePaths(final List<String> deltaFilePaths) {
|
||||
this.deltaFilePaths = deltaFilePaths;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return this.fileId;
|
||||
}
|
||||
|
||||
public void setFileId(final String fileId) {
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return this.partitionPath;
|
||||
}
|
||||
|
||||
public void setPartitionPath(final String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public String getBootstrapFilePath() {
|
||||
return this.bootstrapFilePath;
|
||||
}
|
||||
|
||||
public void setBootstrapFilePath(final String bootstrapFilePath) {
|
||||
this.bootstrapFilePath = bootstrapFilePath;
|
||||
}
|
||||
|
||||
public int getVersion() {
|
||||
return this.version;
|
||||
}
|
||||
|
||||
public void setVersion(final int version) {
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
final ClusteringOperation that = (ClusteringOperation) o;
|
||||
return getVersion() == that.getVersion()
|
||||
&& Objects.equals(getDataFilePath(), that.getDataFilePath())
|
||||
&& Objects.equals(getDeltaFilePaths(), that.getDeltaFilePaths())
|
||||
&& Objects.equals(getFileId(), that.getFileId())
|
||||
&& Objects.equals(getPartitionPath(), that.getPartitionPath())
|
||||
&& Objects.equals(getBootstrapFilePath(), that.getBootstrapFilePath());
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(getDataFilePath(), getDeltaFilePaths(), getFileId(), getPartitionPath(), getBootstrapFilePath(), getVersion());
|
||||
}
|
||||
}
|
||||
@@ -76,6 +76,8 @@ public enum WriteOperationType {
|
||||
return INSERT_OVERWRITE;
|
||||
case "insert_overwrite_table":
|
||||
return INSERT_OVERWRITE_TABLE;
|
||||
case "cluster":
|
||||
return CLUSTER;
|
||||
default:
|
||||
throw new HoodieException("Invalid value of Type.");
|
||||
}
|
||||
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.table.log;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.SpillableMapUtils;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
|
||||
import java.io.IOException;
|
||||
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 static <R extends IndexedRecord, T extends HoodieRecordPayload> HoodieFileSliceReader getFileSliceReader(
|
||||
HoodieFileReader<R> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass) throws IOException {
|
||||
Iterator<R> baseIterator = baseFileReader.getRecordIterator(schema);
|
||||
while (baseIterator.hasNext()) {
|
||||
GenericRecord record = (GenericRecord) baseIterator.next();
|
||||
HoodieRecord<T> hoodieRecord = SpillableMapUtils.convertToHoodieRecordPayload(record, payloadClass);
|
||||
scanner.processNextRecord(hoodieRecord);
|
||||
}
|
||||
return new HoodieFileSliceReader(scanner.iterator());
|
||||
}
|
||||
|
||||
private HoodieFileSliceReader(Iterator<HoodieRecord<? extends HoodieRecordPayload>> recordsItr) {
|
||||
this.recordsIterator = recordsItr;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return recordsIterator.hasNext();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieRecord<? extends HoodieRecordPayload> next() {
|
||||
return recordsIterator.next();
|
||||
}
|
||||
}
|
||||
@@ -306,7 +306,23 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
/**
|
||||
* Transition Clean State from inflight to Committed.
|
||||
* Transition replace requested file to replace inflight.
|
||||
*
|
||||
* @param requestedInstant Requested instant
|
||||
* @param data Extra Metadata
|
||||
* @return inflight instant
|
||||
*/
|
||||
public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option<byte[]> data) {
|
||||
ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
ValidationUtils.checkArgument(requestedInstant.isRequested());
|
||||
HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, requestedInstant.getTimestamp());
|
||||
// Then write to timeline
|
||||
transitionState(requestedInstant, inflightInstant, data);
|
||||
return inflightInstant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Transition replace inflight to Committed.
|
||||
*
|
||||
* @param inflightInstant Inflight instant
|
||||
* @param data Extra Metadata
|
||||
@@ -321,6 +337,26 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
return commitInstant;
|
||||
}
|
||||
|
||||
/**
|
||||
* Revert replace requested State from inflight to requested.
|
||||
*
|
||||
* @param inflightInstant Inflight Instant
|
||||
* @return requested instant
|
||||
*/
|
||||
public HoodieInstant revertReplaceCommitInflightToRequested(HoodieInstant inflightInstant) {
|
||||
ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION));
|
||||
ValidationUtils.checkArgument(inflightInstant.isInflight());
|
||||
HoodieInstant requestedInstant =
|
||||
new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp());
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
|
||||
transitionState(inflightInstant, requestedInstant, Option.empty());
|
||||
} else {
|
||||
deleteInflight(inflightInstant);
|
||||
}
|
||||
return requestedInstant;
|
||||
}
|
||||
|
||||
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
|
||||
transitionState(fromInstant, toInstant, data, false);
|
||||
}
|
||||
|
||||
@@ -299,6 +299,14 @@ public interface HoodieTimeline extends Serializable {
|
||||
return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp);
|
||||
}
|
||||
|
||||
static HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) {
|
||||
return new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, timestamp);
|
||||
}
|
||||
|
||||
static HoodieInstant getReplaceCommitInflightInstant(final String timestamp) {
|
||||
return new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, timestamp);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names
|
||||
* between inflight and completed instants (compaction <=> commit).
|
||||
|
||||
@@ -114,7 +114,11 @@ public class ClusteringUtils {
|
||||
new AbstractMap.SimpleEntry<>(entry.getLeft(), entry.getRight()));
|
||||
}
|
||||
|
||||
private static Stream<HoodieFileGroupId> getFileGroupsFromClusteringGroup(HoodieClusteringGroup group) {
|
||||
public static Stream<HoodieFileGroupId> getFileGroupsFromClusteringPlan(HoodieClusteringPlan clusteringPlan) {
|
||||
return clusteringPlan.getInputGroups().stream().flatMap(ClusteringUtils::getFileGroupsFromClusteringGroup);
|
||||
}
|
||||
|
||||
public static Stream<HoodieFileGroupId> getFileGroupsFromClusteringGroup(HoodieClusteringGroup group) {
|
||||
return group.getSlices().stream().map(slice -> new HoodieFileGroupId(slice.getPartitionPath(), slice.getFileId()));
|
||||
}
|
||||
|
||||
|
||||
@@ -25,6 +25,8 @@ import javax.annotation.Nullable;
|
||||
*/
|
||||
public class StringUtils {
|
||||
|
||||
public static final String EMPTY_STRING = "";
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Joins the elements of the provided array into a single String containing the provided list of elements.
|
||||
|
||||
@@ -0,0 +1,68 @@
|
||||
/*
|
||||
* 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.util;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.config.SerializableSchema;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.ObjectInputStream;
|
||||
import java.io.ObjectOutputStream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Tests serializable schema.
|
||||
*/
|
||||
public class TestSerializableSchema {
|
||||
|
||||
@Test
|
||||
public void testSerDeser() throws IOException {
|
||||
verifySchema(HoodieTestDataGenerator.AVRO_TRIP_SCHEMA);
|
||||
verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_TRIP_SCHEMA));
|
||||
verifySchema(HoodieTestDataGenerator.AVRO_SHORT_TRIP_SCHEMA);
|
||||
verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.AVRO_SHORT_TRIP_SCHEMA));
|
||||
verifySchema(HoodieTestDataGenerator.FLATTENED_AVRO_SCHEMA);
|
||||
verifySchema(HoodieAvroUtils.addMetadataFields(HoodieTestDataGenerator.FLATTENED_AVRO_SCHEMA));
|
||||
verifySchema(HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS);
|
||||
}
|
||||
|
||||
private void verifySchema(Schema schema) throws IOException {
|
||||
SerializableSchema serializableSchema = new SerializableSchema(schema);
|
||||
assertEquals(schema, serializableSchema.get());
|
||||
assertTrue(schema != serializableSchema.get());
|
||||
|
||||
ByteArrayOutputStream baos = new ByteArrayOutputStream();
|
||||
ObjectOutputStream oos = new ObjectOutputStream(baos);
|
||||
serializableSchema.writeObjectTo(oos);
|
||||
oos.flush();
|
||||
oos.close();
|
||||
|
||||
byte[] bytesWritten = baos.toByteArray();
|
||||
SerializableSchema newSchema = new SerializableSchema();
|
||||
newSchema.readObjectFrom(new ObjectInputStream(new ByteArrayInputStream(bytesWritten)));
|
||||
assertEquals(schema, newSchema.get());
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user