1
0

[HUDI-1190] Introduce @PublicAPIClass and @PublicAPIMethod annotations to mark public APIs (#1965)

- Maturity levels one of : evolving, stable, deprecated
- Took a pass and marked out most of the existing public API
This commit is contained in:
vinoth chandar
2020-08-13 23:28:17 -07:00
committed by GitHub
parent 379cf0786f
commit 9bde6d616c
11 changed files with 171 additions and 0 deletions

View File

@@ -22,12 +22,17 @@ import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.PublicAPIClass;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.exception.HoodieException;
/**
* Provide the initial checkpoint for delta streamer.
*/
@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
public abstract class InitialCheckPointProvider {
protected transient Path path;
protected transient FileSystem fs;
@@ -62,5 +67,6 @@ public abstract class InitialCheckPointProvider {
/**
* Get checkpoint string recognizable for delta streamer.
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public abstract String getCheckpoint() throws HoodieException;
}

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.utilities.schema;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.PublicAPIClass;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.avro.Schema;
@@ -28,6 +31,7 @@ import java.io.Serializable;
/**
* Class to provide schema for reading data and also writing into a Hoodie table.
*/
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
public abstract class SchemaProvider implements Serializable {
protected TypedProperties config;
@@ -39,8 +43,10 @@ public abstract class SchemaProvider implements Serializable {
this.jssc = jssc;
}
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public abstract Schema getSourceSchema();
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
public Schema getTargetSchema() {
// by default, use source schema as target for hoodie table as well
return getSourceSchema();

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.utilities.sources;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.PublicAPIClass;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.utilities.schema.SchemaProvider;
@@ -30,6 +33,7 @@ import java.io.Serializable;
/**
* Represents a source from which we can tail data. Assumes a constructor that takes properties.
*/
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
public abstract class Source<T> implements Serializable {
public enum SourceType {
@@ -57,6 +61,7 @@ public abstract class Source<T> implements Serializable {
this.sourceType = sourceType;
}
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
protected abstract InputBatch<T> fetchNewData(Option<String> lastCkptStr, long sourceLimit);
/**

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.utilities.transform;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.PublicAPIClass;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.spark.api.java.JavaSparkContext;
@@ -28,6 +31,7 @@ import org.apache.spark.sql.SparkSession;
/**
* Transform source to target dataset before writing.
*/
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
public interface Transformer {
/**
@@ -39,5 +43,6 @@ public interface Transformer {
* @param properties Config properties
* @return Transformed Dataset
*/
@PublicAPIMethod(maturity = ApiMaturityLevel.STABLE)
Dataset apply(JavaSparkContext jsc, SparkSession sparkSession, Dataset<Row> rowDataset, TypedProperties properties);
}