Skip to content

Commit

Permalink
[CARBONDATA-2961] Simplify SDK API interfaces
Browse files Browse the repository at this point in the history
[CARBONDATA-2961] Simplify SDK API interfaces

problem: current SDK API interfaces are not simpler and don't follow builder pattern.
If new features are added, it will become more complex.

Solution: Simplify the SDK interfaces as per builder pattern.

Refer the latest sdk-guide.

Added:

changes in Carbon Writer:
public CarbonWriterBuilder withThreadSafe(short numOfThreads)
public CarbonWriterBuilder withHadoopConf(Configuration conf)

public CarbonWriterBuilder withCsvInput(Schema schema)
public CarbonWriterBuilder withAvroInput(org.apache.avro.Schema avroSchema)
public CarbonWriterBuilder withJsonInput(Schema carbonSchema)

public CarbonWriter build() throws IOException, InvalidLoadOptionException

Changes in carbon Reader
public CarbonReaderBuilder withHadoopConf(Configuration conf)
public CarbonWriter build() throws IOException, InvalidLoadOptionException

Removed:

changes in Carbon Writer:
public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable)

public CarbonWriterBuilder persistSchemaFile(boolean persist)

setAccessKey
setAccessKey
setSecretKey
setSecretKey
setEndPoint
setEndPoint

public CarbonWriter buildWriterForCSVInput(Schema schema, Configuration configuration)
public CarbonWriter buildThreadSafeWriterForCSVInput(Schema schema, short numOfThreads,Configuration configuration)
public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema avroSchema,Configuration configuration)
public CarbonWriter buildThreadSafeWriterForAvroInput(org.apache.avro.Schema avroSchema,short numOfThreads, Configuration configuration)
public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema, Configuration configuration)
public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, short numOfThreads,Configuration configuration)

Changes in carbon Reader
public CarbonReaderBuilder isTransactionalTable(boolean isTransactionalTable)
public CarbonWriter build(Configuration conf) throws IOException, InvalidLoadOptionException

This closes #2961
  • Loading branch information
ajantha-bhat authored and ravipesala committed Sep 25, 2018
1 parent 1d4d240 commit 6a2a94d
Show file tree
Hide file tree
Showing 28 changed files with 486 additions and 1,793 deletions.
208 changes: 49 additions & 159 deletions docs/sdk-guide.md
Expand Up @@ -67,9 +67,9 @@ These SDK writer output contains just a carbondata and carbonindex files. No met

CarbonProperties.getInstance().addProperty("enable.offheap.sort", enableOffheap);

CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path);
CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path).withCsvInput(schema);

CarbonWriter writer = builder.buildWriterForCSVInput(schema);
CarbonWriter writer = builder.build();

int rows = 5;
for (int i = 0; i < rows; i++) {
Expand Down Expand Up @@ -124,7 +124,7 @@ public class TestSdkAvro {
try {
CarbonWriter writer = CarbonWriter.builder()
.outputPath(path)
.buildWriterForAvroInput(new org.apache.avro.Schema.Parser().parse(avroSchema));
.withAvroInput(new org.apache.avro.Schema.Parser().parse(avroSchema)).build();

for (int i = 0; i < 100; i++) {
writer.write(record);
Expand Down Expand Up @@ -164,10 +164,10 @@ public class TestSdkJson {

Schema CarbonSchema = new Schema(fields);

CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path);
CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path).withJsonInput(CarbonSchema);

// initialize json writer with carbon schema
CarbonWriter writer = builder.buildWriterForJsonInput(CarbonSchema);
CarbonWriter writer = builder.build();
// one row of json Data as String
String JsonRow = "{\"name\":\"abcd\", \"age\":10}";

Expand Down Expand Up @@ -199,7 +199,7 @@ Each of SQL data types are mapped into data types of SDK. Following are the mapp
| DECIMAL | DataTypes.createDecimalType(precision, scale) |

**NOTE:**
Carbon Supports below logical types of AVRO.
1. Carbon Supports below logical types of AVRO.
a. Date
The date logical type represents a date within the calendar, with no reference to a particular time zone or time of day.
A date logical type annotates an Avro int, where the int stores the number of days from the unix epoch, 1 January 1970 (ISO calendar).
Expand All @@ -211,7 +211,9 @@ Each of SQL data types are mapped into data types of SDK. Following are the mapp
A timestamp-micros logical type annotates an Avro long, where the long stores the number of microseconds from the unix epoch, 1 January 1970 00:00:00.000000 UTC.

Currently the values of logical types are not validated by carbon.
Expect that avro record passed by the user is already validated by avro record generator tools.
Expect that avro record passed by the user is already validated by avro record generator tools.
2. If the string data is more than 32K in length, use withTableProperties() with "long_string_columns" property
or directly use DataTypes.VARCHAR if it is carbon schema.

## Run SQL on files directly
Instead of creating table and query it, you can also query that file directly with SQL.
Expand All @@ -234,20 +236,6 @@ Find example code at [DirectSQLExample](https://github.com/apache/carbondata/blo
public CarbonWriterBuilder outputPath(String path);
```

```
/**
* If set false, writes the carbondata and carbonindex files in a flat folder structure
* @param isTransactionalTable is a boolelan value
* if set to false, then writes the carbondata and carbonindex files
* in a flat folder structure.
* if set to true, then writes the carbondata and carbonindex files
* in segment folder structure..
* By default set to false.
* @return updated CarbonWriterBuilder
*/
public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable);
```

```
/**
* to set the timestamp in the carbondata and carbonindex index files
Expand Down Expand Up @@ -306,16 +294,6 @@ public CarbonWriterBuilder localDictionaryThreshold(int localDictionaryThreshold
public CarbonWriterBuilder sortBy(String[] sortColumns);
```

```
/**
* If set, create a schema file in metadata folder.
* @param persist is a boolean value, If set to true, creates a schema file in metadata folder.
* By default set to false. will not create metadata folder
* @return updated CarbonWriterBuilder
*/
public CarbonWriterBuilder persistSchemaFile(boolean persist);
```

```
/**
* sets the taskNo for the writer. SDKs concurrently running
Expand Down Expand Up @@ -370,6 +348,7 @@ public CarbonWriterBuilder withLoadOptions(Map<String, String> options);
* c. local_dictionary_threshold -- positive value, default is 10000
* d. local_dictionary_enable -- true / false. Default is false
* e. sort_columns -- comma separated column. "c1,c2". Default all dimensions are sorted.
If empty string "" is passed. No columns are sorted
* j. sort_scope -- "local_sort", "no_sort", "batch_sort". default value is "local_sort"
* k. long_string_columns -- comma separated string columns which are more than 32k length.
* default value is null.
Expand All @@ -379,91 +358,69 @@ public CarbonWriterBuilder withLoadOptions(Map<String, String> options);
public CarbonWriterBuilder withTableProperties(Map<String, String> options);
```


```
/**
* this writer is not thread safe, use buildThreadSafeWriterForCSVInput in multi thread environment
* Build a {@link CarbonWriter}, which accepts row in CSV format object
* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
* @param configuration hadoop configuration object.
* @return CSVCarbonWriter
* @throws IOException
* @throws InvalidLoadOptionException
* To make sdk writer thread safe.
*
* @param numOfThreads should number of threads in which writer is called in multi-thread scenario
* default sdk writer is not thread safe.
* can use one writer instance in one thread only.
* @return updated CarbonWriterBuilder
*/
public CarbonWriter buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema schema, Configuration configuration) throws IOException, InvalidLoadOptionException;
public CarbonWriterBuilder withThreadSafe(short numOfThreads);
```

```
/**
* Can use this writer in multi-thread instance.
* Build a {@link CarbonWriter}, which accepts row in CSV format
* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
* @param numOfThreads number of threads() in which .write will be called.
* @param configuration hadoop configuration object
* @return CSVCarbonWriter
* @throws IOException
* @throws InvalidLoadOptionException
* To support hadoop configuration
*
* @param conf hadoop configuration support, can set s3a AK,SK,end point and other conf with this
* @return updated CarbonWriterBuilder
*/
public CarbonWriter buildThreadSafeWriterForCSVInput(Schema schema, short numOfThreads, Configuration configuration)
throws IOException, InvalidLoadOptionException;
public CarbonWriterBuilder withHadoopConf(Configuration conf)
```


```
```
/**
* this writer is not thread safe, use buildThreadSafeWriterForAvroInput in multi thread environment
* Build a {@link CarbonWriter}, which accepts Avro format object
* @param avroSchema avro Schema object {org.apache.avro.Schema}
* @param configuration hadoop configuration object
* @return AvroCarbonWriter
* @throws IOException
* @throws InvalidLoadOptionException
* to build a {@link CarbonWriter}, which accepts row in CSV format
*
* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
* @return CarbonWriterBuilder
*/
public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema, Configuration configuration) throws IOException, InvalidLoadOptionException;
public CarbonWriterBuilder withCsvInput(Schema schema);
```

```
/**
* Can use this writer in multi-thread instance.
* Build a {@link CarbonWriter}, which accepts Avro object
* to build a {@link CarbonWriter}, which accepts Avro object
*
* @param avroSchema avro Schema object {org.apache.avro.Schema}
* @param numOfThreads number of threads() in which .write will be called.
* @param configuration hadoop configuration object
* @return AvroCarbonWriter
* @throws IOException
* @throws InvalidLoadOptionException
* @return CarbonWriterBuilder
*/
public CarbonWriter buildThreadSafeWriterForAvroInput(org.apache.avro.Schema avroSchema, short
numOfThreads, Configuration configuration)
throws IOException, InvalidLoadOptionException
public CarbonWriterBuilder withAvroInput(org.apache.avro.Schema avroSchema);
```


```
/**
* this writer is not thread safe, use buildThreadSafeWriterForJsonInput in multi thread environment
* Build a {@link CarbonWriter}, which accepts Json object
* to build a {@link CarbonWriter}, which accepts Json object
*
* @param carbonSchema carbon Schema object
* @param configuration hadoop configuration object
* @return JsonCarbonWriter
* @throws IOException
* @throws InvalidLoadOptionException
* @return CarbonWriterBuilder
*/
public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema, Configuration configuration);
public CarbonWriterBuilder withJsonInput(Schema carbonSchema);
```

```
/**
* Can use this writer in multi-thread instance.
* Build a {@link CarbonWriter}, which accepts Json object
* @param carbonSchema carbon Schema object
* @param numOfThreads number of threads() in which .write will be called.
* @param configuration hadoop configuraiton object.
* @return JsonCarbonWriter
* Build a {@link CarbonWriter}
* This writer is not thread safe,
* use withThreadSafe() configuration in multi thread environment
*
* @return CarbonWriter {AvroCarbonWriter/CSVCarbonWriter/JsonCarbonWriter based on Input Type }
* @throws IOException
* @throws InvalidLoadOptionException
*/
public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, short numOfThreads, Configuration configuration)
public CarbonWriter build() throws IOException, InvalidLoadOptionException;
```

### Class org.apache.carbondata.sdk.file.CarbonWriter
Expand All @@ -474,7 +431,6 @@ public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, s
* which is one row of data.
* If CSVCarbonWriter, object is of type String[], which is one row of data
* If JsonCarbonWriter, object is of type String, which is one row of json
* Note: This API is not thread safe if writer is not built with number of threads argument.
* @param object
* @throws IOException
*/
Expand Down Expand Up @@ -647,19 +603,6 @@ Find example code at [CarbonReaderExample](https://github.com/apache/carbondata/
public CarbonReaderBuilder projection(String[] projectionColumnNames);
```

```
/**
* Configure the transactional status of table
* If set to false, then reads the carbondata and carbonindex files from a flat folder structure.
* If set to true, then reads the carbondata and carbonindex files from segment folder structure.
* Default value is false
*
* @param isTransactionalTable whether is transactional table or not
* @return CarbonReaderBuilder object
*/
public CarbonReaderBuilder isTransactionalTable(boolean isTransactionalTable);
```

```
/**
* Configure the filter expression for carbon reader
Expand All @@ -671,66 +614,13 @@ Find example code at [CarbonReaderExample](https://github.com/apache/carbondata/
```

```
/**
* Set the access key for S3
*
* @param key the string of access key for different S3 type,like: fs.s3a.access.key
* @param value the value of access key
* @return CarbonWriterBuilder
*/
public CarbonReaderBuilder setAccessKey(String key, String value);
```

```
/**
* Set the access key for S3.
*
* @param value the value of access key
* @return CarbonWriterBuilder object
*/
public CarbonReaderBuilder setAccessKey(String value);
```

```
/**
* Set the secret key for S3
*
* @param key the string of secret key for different S3 type,like: fs.s3a.secret.key
* @param value the value of secret key
* @return CarbonWriterBuilder object
*/
public CarbonReaderBuilder setSecretKey(String key, String value);
```

```
/**
* Set the secret key for S3
*
* @param value the value of secret key
* @return CarbonWriterBuilder object
*/
public CarbonReaderBuilder setSecretKey(String value);
```

```
/**
* Set the endpoint for S3
*
* @param key the string of endpoint for different S3 type,like: fs.s3a.endpoint
* @param value the value of endpoint
* @return CarbonWriterBuilder object
*/
public CarbonReaderBuilder setEndPoint(String key, String value);
```

```
/**
* Set the endpoint for S3
*
* @param value the value of endpoint
* @return CarbonWriterBuilder object
*/
public CarbonReaderBuilder setEndPoint(String value);
/**
* To support hadoop configuration
*
* @param conf hadoop configuration support, can set s3a AK,SK,end point and other conf with this
* @return updated CarbonReaderBuilder
*/
public CarbonReaderBuilder withHadoopConf(Configuration conf);
```

```
Expand Down
Expand Up @@ -25,7 +25,6 @@
import java.util.Map;

import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;

import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.sdk.file.CarbonReader;
Expand Down Expand Up @@ -62,7 +61,7 @@ public static void main(String[] args) {
CarbonWriter writer = CarbonWriter.builder()
.outputPath(path)
.withLoadOptions(map)
.buildWriterForCSVInput(new Schema(fields), new Configuration(false));
.withCsvInput(new Schema(fields)).build();

for (int i = 0; i < 10; i++) {
String[] row2 = new String[]{
Expand Down Expand Up @@ -107,7 +106,7 @@ public boolean accept(File dir, String name) {
CarbonReader reader = CarbonReader
.builder(path, "_temp")
.projection(strings)
.build(new Configuration(false));
.build();

System.out.println("\nData:");
long day = 24L * 3600 * 1000;
Expand All @@ -132,7 +131,7 @@ public boolean accept(File dir, String name) {
// Read data
CarbonReader reader2 = CarbonReader
.builder(path, "_temp")
.build(new Configuration(false));
.build();

System.out.println("\nData:");
i = 0;
Expand Down

0 comments on commit 6a2a94d

Please sign in to comment.