From 6d08a5261037c762ea7b5297a22840c00689e890 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Wed, 1 Aug 2018 02:16:26 +0800 Subject: [PATCH 01/14] modify CarbonStore API --- .../annotations/InterfaceStability.java | 2 +- .../carbondata/store/WorkerManager.scala | 4 +- .../org/apache/spark/sql/CarbonSession.scala | 16 +- .../store/impl/CarbonStoreBase.java | 39 +++-- .../store/impl/DistributedCarbonStore.java | 59 ++++--- .../store/impl/LocalCarbonStore.java | 61 +++++--- .../carbondata/store/impl/MetaProcessor.java | 25 +-- .../carbondata/store/impl/master/Master.java | 4 +- .../store/impl/master/Scheduler.java | 4 +- .../store/impl/rpc/model/LoadDataRequest.java | 2 +- .../store/impl/worker/RequestHandler.java | 4 +- .../carbondata/store/impl/worker/Worker.java | 4 +- .../store/DistributedCarbonStoreTest.java | 26 ++-- .../store/LocalCarbonStoreTest.java | 27 ++-- .../org/apache/carbondata/store/TestUtil.java | 10 -- .../carbondata/horizon/HorizonTest.java | 2 +- store/sdk/pom.xml | 4 +- .../carbondata/sdk/store/CarbonStore.java | 146 ++++++++++++++++++ .../sdk/store}/CarbonStoreFactory.java | 21 ++- .../carbondata/sdk/store/ColumnVector.java} | 9 +- .../carbondata/sdk/store/ColumnarBatch.java | 27 ++++ .../apache/carbondata/sdk/store/Fetcher.java | 37 +++++ .../carbondata/sdk/store/KeyedRow.java} | 42 +++-- .../apache/carbondata/sdk/store/Loader.java | 47 ++++++ .../apache/carbondata/sdk/store/Mutator.java} | 34 ++-- .../carbondata/sdk/store/PrimaryKey.java} | 19 +-- .../carbondata/sdk/store/ResultBatch.java | 47 ++++++ .../org/apache/carbondata/sdk/store/Row.java | 27 ++++ .../apache/carbondata/sdk/store/ScanUnit.java | 40 +++++ .../apache/carbondata/sdk/store/Scanner.java | 88 +++++++++++ .../carbondata/sdk/store/SelectOption.java | 28 ++++ .../carbondata/sdk/store/SqlOperation.java} | 4 +- .../sdk/store/TransactionalOperation.java | 35 +++++ .../carbondata/sdk/store}/conf/StoreConf.java | 4 +- .../sdk/store}/descriptor/LoadDescriptor.java | 7 +- .../store}/descriptor/SelectDescriptor.java | 12 +- .../store}/descriptor/TableDescriptor.java | 2 +- .../store}/descriptor/TableIdentifier.java | 11 +- .../sdk/store/exception/CarbonException.java | 38 +++++ .../exception/ExecutionTimeoutException.java | 7 +- .../store}/exception/SchedulerException.java | 7 +- .../carbondata/sdk}/store/util/StoreUtil.java | 4 +- 42 files changed, 826 insertions(+), 210 deletions(-) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/CarbonStoreFactory.java (85%) rename store/{core/src/main/java/org/apache/carbondata/store/api/CarbonStore.java => sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java} (84%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java rename store/{core/src/main/java/org/apache/carbondata/store/api/DataStore.java => sdk/src/main/java/org/apache/carbondata/sdk/store/KeyedRow.java} (53%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java rename store/{core/src/main/java/org/apache/carbondata/store/api/MetaStore.java => sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java} (53%) rename store/{core/src/main/java/org/apache/carbondata/store/api/exception/StoreException.java => sdk/src/main/java/org/apache/carbondata/sdk/store/PrimaryKey.java} (74%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/Row.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java rename store/{core/src/main/java/org/apache/carbondata/store/api/SqlStore.java => sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java} (93%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/conf/StoreConf.java (98%) rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/descriptor/LoadDescriptor.java (92%) rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/descriptor/SelectDescriptor.java (86%) rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/descriptor/TableDescriptor.java (98%) rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/descriptor/TableIdentifier.java (77%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/CarbonException.java rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/exception/ExecutionTimeoutException.java (78%) rename store/{core/src/main/java/org/apache/carbondata/store/api => sdk/src/main/java/org/apache/carbondata/sdk/store}/exception/SchedulerException.java (79%) rename store/{core/src/main/java/org/apache/carbondata => sdk/src/main/java/org/apache/carbondata/sdk}/store/util/StoreUtil.java (97%) diff --git a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java index 5435028b7c5..afd863f4ca5 100644 --- a/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java +++ b/common/src/main/java/org/apache/carbondata/common/annotations/InterfaceStability.java @@ -42,7 +42,7 @@ * */ @InterfaceAudience.User -@org.apache.hadoop.classification.InterfaceStability.Evolving +@InterfaceStability.Evolving public class InterfaceStability { /** * Can evolve while retaining compatibility for minor release boundaries.; diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/store/WorkerManager.scala b/integration/spark2/src/main/scala/org/apache/carbondata/store/WorkerManager.scala index 7fff2e565e0..c62470244b6 100644 --- a/integration/spark2/src/main/scala/org/apache/carbondata/store/WorkerManager.scala +++ b/integration/spark2/src/main/scala/org/apache/carbondata/store/WorkerManager.scala @@ -25,9 +25,9 @@ import org.apache.spark.sql.SparkSession import org.apache.carbondata.common.annotations.InterfaceAudience import org.apache.carbondata.common.logging.LogServiceFactory import org.apache.carbondata.core.util.CarbonProperties +import org.apache.carbondata.sdk.store.CarbonStoreFactory +import org.apache.carbondata.sdk.store.conf.StoreConf import org.apache.carbondata.spark.util.Util -import org.apache.carbondata.store.api.CarbonStoreFactory -import org.apache.carbondata.store.api.conf.StoreConf import org.apache.carbondata.store.impl.worker.Worker /** diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala index 6c139558014..3a93c091e23 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala @@ -22,9 +22,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ -import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} +import org.apache.spark.SparkContext import org.apache.spark.sql.SparkSession.Builder import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression} @@ -35,18 +33,16 @@ import org.apache.spark.sql.hive.execution.command.CarbonSetCommand import org.apache.spark.sql.internal.{SessionState, SharedState} import org.apache.spark.sql.optimizer.CarbonFilters import org.apache.spark.sql.profiler.{Profiler, SQLStart} -import org.apache.spark.util.{CarbonReflectionUtils, Utils} +import org.apache.spark.util.CarbonReflectionUtils import org.apache.carbondata.common.annotations.InterfaceAudience import org.apache.carbondata.common.logging.{LogService, LogServiceFactory} import org.apache.carbondata.core.constants.CarbonCommonConstants import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, ThreadLocalSessionInfo} -import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil +import org.apache.carbondata.sdk.store.{CarbonStore, CarbonStoreFactory} +import org.apache.carbondata.sdk.store.conf.StoreConf +import org.apache.carbondata.sdk.store.descriptor.{SelectDescriptor, TableIdentifier} import org.apache.carbondata.store.WorkerManager -import org.apache.carbondata.store.api.{CarbonStore, CarbonStoreFactory} -import org.apache.carbondata.store.api.conf.StoreConf -import org.apache.carbondata.store.api.descriptor.{SelectDescriptor, TableIdentifier => CTableIdentifier} -import org.apache.carbondata.streaming.CarbonStreamingQueryListener /** * Session implementation for {org.apache.spark.sql.SparkSession} @@ -242,7 +238,7 @@ class CarbonSession(@transient val sc: SparkContext, localMaxRows: Option[Long] = None): DataFrame = { val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable val select = new SelectDescriptor( - new CTableIdentifier(table.getTableName, table.getDatabaseName), + new TableIdentifier(table.getTableName, table.getDatabaseName), columns.map(_.name).toArray, if (expr != null) CarbonFilters.transformExpression(expr) else null, localMaxRows.getOrElse(Long.MaxValue) diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java index 7e50102f686..05e2b3b10de 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java @@ -40,11 +40,11 @@ import org.apache.carbondata.hadoop.api.CarbonInputFormat; import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; -import org.apache.carbondata.store.api.CarbonStore; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.CarbonStore; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.store.impl.rpc.model.Scan; import org.apache.hadoop.conf.Configuration; @@ -71,21 +71,34 @@ public abstract class CarbonStoreBase implements CarbonStore { } @Override - public void createTable(TableDescriptor table) throws IOException, StoreException { - Objects.requireNonNull(table); - metaProcessor.createTable(table); + public void createTable(TableDescriptor descriptor) throws CarbonException { + Objects.requireNonNull(descriptor); + metaProcessor.createTable(descriptor); } @Override - public void dropTable(TableIdentifier table) throws IOException { + public void dropTable(TableIdentifier table) throws CarbonException { Objects.requireNonNull(table); - metaProcessor.dropTable(table); + try { + metaProcessor.dropTable(table); + } catch (IOException e) { + throw new CarbonException(e); + } } @Override - public CarbonTable getTable(TableIdentifier table) throws IOException { - Objects.requireNonNull(table); - return metaProcessor.getTable(table); + public List listTable() throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable(TableIdentifier table, TableDescriptor newTable) throws CarbonException { + throw new UnsupportedOperationException(); } public String getTablePath(String tableName, String databaseName) { diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java index 3667aea899d..e014ff20eb7 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java @@ -44,11 +44,13 @@ import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; import org.apache.carbondata.processing.util.CarbonLoaderUtil; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.SelectDescriptor; -import org.apache.carbondata.store.api.exception.ExecutionTimeoutException; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.Loader; +import org.apache.carbondata.sdk.store.Scanner; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.exception.ExecutionTimeoutException; import org.apache.carbondata.store.impl.master.Schedulable; import org.apache.carbondata.store.impl.master.Scheduler; import org.apache.carbondata.store.impl.rpc.model.BaseResponse; @@ -75,9 +77,14 @@ class DistributedCarbonStore extends CarbonStoreBase { } @Override - public void loadData(LoadDescriptor load) throws IOException, StoreException { + public void loadData(LoadDescriptor load) throws CarbonException { Objects.requireNonNull(load); - CarbonTable table = metaProcessor.getTable(load.getTable()); + CarbonTable table = null; + try { + table = metaProcessor.getTable(load.getTable()); + } catch (IOException e) { + throw new CarbonException(e); + } CarbonLoadModelBuilder builder = new CarbonLoadModelBuilder(table); builder.setInputPath(load.getInputPath()); CarbonLoadModel loadModel; @@ -85,10 +92,10 @@ public void loadData(LoadDescriptor load) throws IOException, StoreException { loadModel = builder.build(load.getOptions(), System.currentTimeMillis(), "0"); } catch (InvalidLoadOptionException e) { LOGGER.error(e, "Invalid loadDescriptor options"); - throw new StoreException(e); + throw new CarbonException(e); } catch (IOException e) { LOGGER.error(e, "Failed to loadDescriptor data"); - throw e; + throw new CarbonException(e); } Schedulable worker = scheduler.pickNexWorker(); @@ -103,23 +110,39 @@ public void loadData(LoadDescriptor load) throws IOException, StoreException { txnManager.commitSegment(loadModel); } else { txnManager.closeSegment(loadModel); - throw new StoreException(response.getMessage()); + throw new CarbonException(response.getMessage()); } + } catch (IOException e) { + throw new CarbonException(e); } finally { worker.workload.decrementAndGet(); } } @Override - public List select(SelectDescriptor select) throws IOException, StoreException { + public Loader newLoader(LoadDescriptor load) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public List select(SelectDescriptor select) throws CarbonException { Objects.requireNonNull(select); - CarbonTable carbonTable = metaProcessor.getTable(select.getTable()); - return select( - carbonTable, - select.getProjection(), - select.getFilter(), - select.getLimit(), - select.getLimit()); + try { + CarbonTable carbonTable = metaProcessor.getTable(select.getTableIdentifier()); + return select( + carbonTable, + select.getProjection(), + select.getFilter(), + select.getLimit(), + select.getLimit()); + } catch (IOException e) { + throw new CarbonException(e); + } + } + + @Override + public Scanner newScanner() throws CarbonException { + throw new UnsupportedOperationException(); } /** diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index 40b6bdffc4e..b858fd4d242 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -40,12 +40,14 @@ import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.SelectDescriptor; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.Loader; +import org.apache.carbondata.sdk.store.Scanner; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.rpc.model.Scan; -import org.apache.carbondata.store.util.StoreUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; @@ -84,17 +86,19 @@ class LocalCarbonStore extends CarbonStoreBase { } @Override - public void loadData(LoadDescriptor load) throws IOException, StoreException { + public void loadData(LoadDescriptor load) throws CarbonException { Objects.requireNonNull(load); - CarbonTable table = metaProcessor.getTable(load.getTable()); - CarbonLoadModelBuilder modelBuilder = new CarbonLoadModelBuilder(table); - modelBuilder.setInputPath(load.getInputPath()); CarbonLoadModel loadModel; try { + CarbonTable table = metaProcessor.getTable(load.getTable()); + CarbonLoadModelBuilder modelBuilder = new CarbonLoadModelBuilder(table); + modelBuilder.setInputPath(load.getInputPath()); loadModel = modelBuilder.build(load.getOptions(), System.currentTimeMillis(), "0"); } catch (InvalidLoadOptionException e) { LOGGER.error(e, "Invalid loadDescriptor options"); - throw new StoreException(e.getMessage()); + throw new CarbonException(e); + } catch (IOException e) { + throw new CarbonException(e); } if (loadModel.getFactTimeStamp() == 0) { @@ -106,12 +110,22 @@ public void loadData(LoadDescriptor load) throws IOException, StoreException { loadData(loadModel); txnManager.commitSegment(loadModel); } catch (Exception e) { - txnManager.closeSegment(loadModel); LOGGER.error(e, "Failed to load data"); - throw new StoreException(e); + try { + txnManager.closeSegment(loadModel); + } catch (IOException ex) { + LOGGER.error(ex, "Failed to close segment"); + // Ignoring the exception + } + throw new CarbonException(e); } } + @Override + public Loader newLoader(LoadDescriptor load) throws CarbonException { + throw new UnsupportedOperationException(); + } + private void loadData(CarbonLoadModel model) throws Exception { DataLoadExecutor executor = null; try { @@ -146,15 +160,22 @@ private void loadData(CarbonLoadModel model) throws Exception { } @Override - public List select(SelectDescriptor select) throws IOException { + public List select(SelectDescriptor select) throws CarbonException { Objects.requireNonNull(select); - CarbonTable table = metaProcessor.getTable(select.getTable()); - List blocks = pruneBlock(table, select.getProjection(), select.getFilter()); - CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, ""); - Scan scan = new Scan( - 0, split, table.getTableInfo(), select.getProjection(), select.getFilter(), - select.getLimit()); - return scan(table, scan); + try { + CarbonTable table = metaProcessor.getTable(select.getTableIdentifier()); + List blocks = pruneBlock(table, select.getProjection(), select.getFilter()); + CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, ""); + Scan scan = new Scan(0, split, table.getTableInfo(), select.getProjection(), + select.getFilter(), select.getLimit()); + return scan(table, scan); + } catch (IOException e) { + throw new CarbonException(e); + } + } + + @Override public Scanner newScanner() throws CarbonException { + return null; } @Override diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java b/store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java index 6d037116f85..a2dd95525b5 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java @@ -42,9 +42,9 @@ import org.apache.carbondata.core.writer.ThriftWriter; import org.apache.carbondata.sdk.file.CarbonWriterBuilder; import org.apache.carbondata.sdk.file.Field; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; class MetaProcessor { @@ -60,21 +60,23 @@ class MetaProcessor { // mapping of table path to CarbonTable object private Map cache = new HashMap<>(); - public void createTable(TableDescriptor descriptor) throws StoreException { + public void createTable(TableDescriptor descriptor) throws + CarbonException { + TableIdentifier table = descriptor.getTable(); Field[] fields = descriptor.getSchema().getFields(); // sort_columns List sortColumnsList = null; try { sortColumnsList = descriptor.getSchema().prepareSortColumns(descriptor.getProperties()); } catch (MalformedCarbonCommandException e) { - throw new StoreException(e.getMessage()); + throw new CarbonException(e.getMessage()); } ColumnSchema[] sortColumnsSchemaList = new ColumnSchema[sortColumnsList.size()]; TableSchemaBuilder builder = TableSchema.builder(); CarbonWriterBuilder.buildTableSchema(fields, builder, sortColumnsList, sortColumnsSchemaList); - TableSchema schema = builder.tableName(descriptor.getTable().getTableName()) + TableSchema schema = builder.tableName(table.getTableName()) .properties(descriptor.getProperties()) .setSortColumns(Arrays.asList(sortColumnsSchemaList)) .build(); @@ -82,17 +84,16 @@ public void createTable(TableDescriptor descriptor) throws StoreException { SchemaEvolutionEntry schemaEvolutionEntry = new SchemaEvolutionEntry(); schemaEvolutionEntry.setTimeStamp(System.currentTimeMillis()); schema.getSchemaEvolution().getSchemaEvolutionEntryList().add(schemaEvolutionEntry); - schema.setTableName(descriptor.getTable().getTableName()); + schema.setTableName(table.getTableName()); String tablePath = descriptor.getTablePath(); if (tablePath == null) { - tablePath = store.getTablePath( - descriptor.getTable().getTableName(), descriptor.getTable().getDatabaseName()); + tablePath = store.getTablePath(table.getTableName(), table.getDatabaseName()); } TableInfo tableInfo = CarbonTable.builder() - .databaseName(descriptor.getTable().getDatabaseName()) - .tableName(descriptor.getTable().getTableName()) + .databaseName(table.getDatabaseName()) + .tableName(table.getTableName()) .tablePath(tablePath) .tableSchema(schema) .isTransactionalTable(true) @@ -102,7 +103,7 @@ public void createTable(TableDescriptor descriptor) throws StoreException { createTable(tableInfo, descriptor.isIfNotExists()); } catch (IOException e) { LOGGER.error(e, "create tableDescriptor failed"); - throw new StoreException(e.getMessage()); + throw new CarbonException(e.getMessage()); } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java index 59d4aa384b1..f52592375c7 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java @@ -25,13 +25,13 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.store.api.conf.StoreConf; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.rpc.RegistryService; import org.apache.carbondata.store.impl.rpc.ServiceFactory; import org.apache.carbondata.store.impl.rpc.StoreService; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; -import org.apache.carbondata.store.util.StoreUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.RPC; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java index 96a137518c2..2817e9d151c 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java @@ -31,8 +31,8 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.exception.SchedulerException; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.exception.SchedulerException; import org.apache.carbondata.store.impl.rpc.model.BaseResponse; import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; import org.apache.carbondata.store.impl.rpc.model.QueryResponse; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java index e6c80488b2c..552c5cb0cb3 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java @@ -23,7 +23,7 @@ import java.io.Serializable; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; -import org.apache.carbondata.store.util.StoreUtil; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java index fd13b20d338..981ab7672a6 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java @@ -36,7 +36,8 @@ import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat; import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; -import org.apache.carbondata.store.api.conf.StoreConf; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.CarbonStoreBase; import org.apache.carbondata.store.impl.Status; import org.apache.carbondata.store.impl.rpc.model.BaseResponse; @@ -45,7 +46,6 @@ import org.apache.carbondata.store.impl.rpc.model.Scan; import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; import org.apache.carbondata.store.impl.rpc.model.ShutdownResponse; -import org.apache.carbondata.store.util.StoreUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java index a360e365d56..101cb276c54 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java @@ -23,13 +23,13 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.store.api.conf.StoreConf; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.rpc.RegistryService; import org.apache.carbondata.store.impl.rpc.ServiceFactory; import org.apache.carbondata.store.impl.rpc.StoreService; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; -import org.apache.carbondata.store.util.StoreUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.RPC; diff --git a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java index 24486604a35..bb62b276d20 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java @@ -27,14 +27,14 @@ import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.LiteralExpression; import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; -import org.apache.carbondata.store.api.CarbonStore; -import org.apache.carbondata.store.api.CarbonStoreFactory; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.SelectDescriptor; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.CarbonStore; +import org.apache.carbondata.sdk.store.CarbonStoreFactory; +import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.store.impl.worker.Worker; import org.junit.After; @@ -50,7 +50,7 @@ public class DistributedCarbonStoreTest { private static CarbonStore store; @BeforeClass - public static void beforeAll() throws IOException, StoreException { + public static void beforeAll() throws IOException, CarbonException { projectFolder = new File(DistributedCarbonStoreTest.class.getResource("/").getPath() + "../../../../").getCanonicalPath(); String confFile = projectFolder + "/store/conf/store.conf"; @@ -81,13 +81,13 @@ public void verifyDMFile() { } @Test - public void testSelect() throws IOException, StoreException { + public void testSelect() throws IOException, CarbonException { TableIdentifier tableIdentifier = new TableIdentifier("table_1", "default"); store.dropTable(tableIdentifier); - TableDescriptor table = TableDescriptor + TableDescriptor descriptor = TableDescriptor .builder() - .ifNotExists() .table(tableIdentifier) + .ifNotExists() .comment("first table") .column("shortField", DataTypes.SHORT, "short field") .column("intField", DataTypes.INT, "int field") @@ -101,7 +101,7 @@ public void testSelect() throws IOException, StoreException { .column("floatField", DataTypes.DOUBLE, "float field") .tblProperties(CarbonCommonConstants.SORT_COLUMNS, "intField") .create(); - store.createTable(table); + store.createTable(descriptor); // load one segment LoadDescriptor load = LoadDescriptor diff --git a/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java index 420c8cf02a6..419c00fac4f 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java @@ -22,20 +22,19 @@ import java.util.List; import org.apache.carbondata.core.constants.CarbonCommonConstants; -import org.apache.carbondata.core.datastore.impl.FileFactory; import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.LiteralExpression; import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; -import org.apache.carbondata.store.api.CarbonStore; -import org.apache.carbondata.store.api.CarbonStoreFactory; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.SelectDescriptor; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.CarbonStore; +import org.apache.carbondata.sdk.store.CarbonStoreFactory; +import org.apache.carbondata.sdk.store.conf.StoreConf; import org.junit.After; import org.junit.AfterClass; @@ -50,7 +49,7 @@ public class LocalCarbonStoreTest { private static CarbonStore store; @BeforeClass - public static void setup() throws IOException, StoreException { + public static void setup() throws IOException, CarbonException { StoreConf conf = new StoreConf("test", "./"); conf.conf(StoreConf.STORE_TEMP_LOCATION, "./temp"); store = CarbonStoreFactory.getLocalStore("LocalCarbonStoreTest", conf); @@ -74,13 +73,13 @@ public void verifyDMFile() { } @Test - public void testWriteAndReadFiles() throws IOException, StoreException { + public void testWriteAndReadFiles() throws IOException, CarbonException { TableIdentifier tableIdentifier = new TableIdentifier("table_1", "default"); store.dropTable(tableIdentifier); - TableDescriptor table = TableDescriptor + TableDescriptor descriptor = TableDescriptor .builder() - .ifNotExists() .table(tableIdentifier) + .ifNotExists() .comment("first table") .column("shortField", DataTypes.SHORT, "short field") .column("intField", DataTypes.INT, "int field") @@ -94,7 +93,7 @@ public void testWriteAndReadFiles() throws IOException, StoreException { .column("floatField", DataTypes.DOUBLE, "float field") .tblProperties(CarbonCommonConstants.SORT_COLUMNS, "intField") .create(); - store.createTable(table); + store.createTable(descriptor); // load one segment LoadDescriptor load = LoadDescriptor diff --git a/store/core/src/test/java/org/apache/carbondata/store/TestUtil.java b/store/core/src/test/java/org/apache/carbondata/store/TestUtil.java index f73591c8679..dffc8a7d477 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/TestUtil.java +++ b/store/core/src/test/java/org/apache/carbondata/store/TestUtil.java @@ -18,21 +18,11 @@ package org.apache.carbondata.store; import java.io.File; -import java.io.FileFilter; import java.io.IOException; -import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException; import org.apache.carbondata.core.constants.CarbonCommonConstants; import org.apache.carbondata.core.datastore.impl.FileFactory; import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.core.util.path.CarbonTablePath; -import org.apache.carbondata.sdk.file.CarbonWriter; -import org.apache.carbondata.sdk.file.CarbonWriterBuilder; -import org.apache.carbondata.sdk.file.Schema; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.util.StoreUtil; - -import org.junit.Assert; public class TestUtil { diff --git a/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java b/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java index 91a9dba9d72..b980cb064f7 100644 --- a/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java +++ b/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java @@ -34,7 +34,7 @@ import org.apache.carbondata.store.api.conf.StoreConf; import org.apache.carbondata.store.api.exception.StoreException; import org.apache.carbondata.store.impl.worker.Worker; -import org.apache.carbondata.store.util.StoreUtil; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.junit.AfterClass; import org.junit.Assert; diff --git a/store/sdk/pom.xml b/store/sdk/pom.xml index aecf7e28879..b16a290311f 100644 --- a/store/sdk/pom.xml +++ b/store/sdk/pom.xml @@ -51,8 +51,8 @@ maven-compiler-plugin - 1.7 - 1.7 + 1.8 + 1.8 diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java new file mode 100644 index 00000000000..399802b8341 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -0,0 +1,146 @@ +/* + * 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.carbondata.sdk.store; + +import java.io.Closeable; +import java.util.List; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +/** + * Public Interface of CarbonStore + */ +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface CarbonStore extends Closeable { + + //////////////////////////////////////////////////////////////////// + ///// Metadata Operation ///// + //////////////////////////////////////////////////////////////////// + + /** + * Create a Table + * @param descriptor descriptor for create table operation + * @throws CarbonException if any error occurs + */ + void createTable(TableDescriptor descriptor) throws CarbonException; + + /** + * Drop a Table, and remove all data in it + * @param table table identifier + * @throws CarbonException if any error occurs + */ + void dropTable(TableIdentifier table) throws CarbonException; + + /** + * @return all table created + * @throws CarbonException if any error occurs + */ + List listTable() throws CarbonException; + + /** + * Return table descriptor by specified identifier + * @param table table identifier + * @return table descriptor + * @throws CarbonException if any error occurs + */ + TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException; + + /** + * Alter table operation + * @param table table identifier + * @param newTable new table descriptor to alter to + * @throws CarbonException if any error occurs + */ + void alterTable(TableIdentifier table, TableDescriptor newTable) throws CarbonException; + + + //////////////////////////////////////////////////////////////////// + ///// Write Operation ///// + //////////////////////////////////////////////////////////////////// + + /** + * Trigger a Load into the table specified by load descriptor + * @param load descriptor for load operation + * @throws CarbonException if any error occurs + */ + void loadData(LoadDescriptor load) throws CarbonException; + + /** + * Create a new Loader, which can be used to load data in distributed compute framework + * @param load descriptor for load operation + * @return a new Loader + * @throws CarbonException if any error occurs + */ + Loader newLoader(LoadDescriptor load) throws CarbonException; + + /** + * Return true if this table has primary key defined when create table using + * {@link #createTable(TableDescriptor)} + * + * For a such table, {@link #newMutator()} and {@link #newFetcher()} are supported + * + * @return true if this table has primary key. + */ + default boolean isPrimaryKeyDefined() { + return false; + } + + /** + * A mutator supports upsert and delete using primary key + * @return a new mutator + * @throws CarbonException if any error occurs + */ + default Mutator newMutator() throws CarbonException { + throw new UnsupportedOperationException(); + } + + + //////////////////////////////////////////////////////////////////// + ///// Read Operation ///// + //////////////////////////////////////////////////////////////////// + + /** + * Scan a Table and return matched rows + * @param select descriptor for scan operation, including required column, filter, etc + * @return matched rows + * @throws CarbonException if any error occurs + */ + List select(SelectDescriptor select) throws CarbonException; + + /** + * @return a new Scanner + * @throws CarbonException if any error occurs + */ + Scanner newScanner() throws CarbonException; + + /** + * @return a new Fetcher + * @throws CarbonException if any error occurs + */ + default Fetcher newFetcher() throws CarbonException { + throw new UnsupportedOperationException(); + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/CarbonStoreFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java similarity index 85% rename from store/core/src/main/java/org/apache/carbondata/store/api/CarbonStoreFactory.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java index 76ef450d23e..761a2113777 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/CarbonStoreFactory.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.api; +package org.apache.carbondata.sdk.store; import java.io.IOException; import java.lang.reflect.Constructor; @@ -23,9 +23,16 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.exception.CarbonException; +/** + * Factory class to create {@link CarbonStore} + */ +@InterfaceAudience.User +@InterfaceStability.Unstable public class CarbonStoreFactory { private static Map distributedStores = new ConcurrentHashMap<>(); private static Map localStores = new ConcurrentHashMap<>(); @@ -34,7 +41,7 @@ private CarbonStoreFactory() { } public static CarbonStore getDistributedStore(String storeName, StoreConf storeConf) - throws StoreException { + throws CarbonException { if (distributedStores.containsKey(storeName)) { return distributedStores.get(storeName); } @@ -47,7 +54,7 @@ public static CarbonStore getDistributedStore(String storeName, StoreConf storeC return store; } catch (ClassNotFoundException | IllegalAccessException | InvocationTargetException | InstantiationException e) { - throw new StoreException(e); + throw new CarbonException(e); } } @@ -59,7 +66,7 @@ public static void removeDistributedStore(String storeName) throws IOException { } public static CarbonStore getLocalStore(String storeName, StoreConf storeConf) - throws StoreException { + throws CarbonException { if (localStores.containsKey(storeName)) { return localStores.get(storeName); } @@ -72,7 +79,7 @@ public static CarbonStore getLocalStore(String storeName, StoreConf storeConf) return store; } catch (ClassNotFoundException | IllegalAccessException | InvocationTargetException | InstantiationException e) { - throw new StoreException(e); + throw new CarbonException(e); } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java similarity index 84% rename from store/core/src/main/java/org/apache/carbondata/store/api/CarbonStore.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java index 3525389df25..174d46d7f4a 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java @@ -15,18 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.store.api; - -import java.io.Closeable; +package org.apache.carbondata.sdk.store; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -/** - * Public Interface of CarbonStore - */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface CarbonStore extends MetaStore, DataStore, Closeable { +public interface ColumnVector { } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java new file mode 100644 index 00000000000..2d42d36dc46 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java @@ -0,0 +1,27 @@ +/* + * 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.carbondata.sdk.store; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface ColumnarBatch { + ColumnVector get(int ordinal); +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java new file mode 100644 index 00000000000..cc26e9a4058 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java @@ -0,0 +1,37 @@ +/* + * 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.carbondata.sdk.store; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +/** + * A Fetcher is used to lookup row by primary key + */ +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface Fetcher { + /** + * Lookup and return a row with specified primary key + * @param key key to lookup + * @return matched row for the specified key + * @throws CarbonException if any error occurs + */ + Row lookup(PrimaryKey key) throws CarbonException; +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/DataStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/KeyedRow.java similarity index 53% rename from store/core/src/main/java/org/apache/carbondata/store/api/DataStore.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/KeyedRow.java index d35c1334e72..bff02e055a6 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/DataStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/KeyedRow.java @@ -15,37 +15,31 @@ * limitations under the License. */ -package org.apache.carbondata.store.api; - -import java.io.IOException; -import java.util.List; +package org.apache.carbondata.sdk.store; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.SelectDescriptor; -import org.apache.carbondata.store.api.exception.StoreException; -/** - * Public interface to write and read data in CarbonStore - */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface DataStore { +public class KeyedRow { + private PrimaryKey key; + private CarbonRow row; + + public void setKey(PrimaryKey key) { + this.key = key; + } + + public void setRow(CarbonRow row) { + this.row = row; + } - /** - * Load data into a Table - * @param load descriptor for load operation - * @throws IOException if network or disk IO error occurs - */ - void loadData(LoadDescriptor load) throws IOException, StoreException; + public PrimaryKey getKey() { + return key; + } - /** - * Scan a Table and return matched rows - * @param select descriptor for scan operation, including required column, filter, etc - * @return matched rows - * @throws IOException if network or disk IO error occurs - */ - List select(SelectDescriptor select) throws IOException, StoreException; + public CarbonRow getRow() { + return row; + } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java new file mode 100644 index 00000000000..0c3e73b7f00 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java @@ -0,0 +1,47 @@ +/* + * 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.carbondata.sdk.store; + +import java.util.Iterator; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.metadata.datatype.StructType; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +/** + * A Loader is used to load data from files to the table + */ +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface Loader extends TransactionalOperation { + /** + * Trigger the load operation + * @throws CarbonException if any error occurs + */ + void load() throws CarbonException; + + /** + * Append a batch of rows. + * @param rows rows to append + * @param schema schema of the input row + * @throws CarbonException if any error occurs + */ + void append(Iterator rows, StructType schema) throws CarbonException; + +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/MetaStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java similarity index 53% rename from store/core/src/main/java/org/apache/carbondata/store/api/MetaStore.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java index dea6873f348..28dc9a2e70a 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/MetaStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java @@ -15,36 +15,34 @@ * limitations under the License. */ -package org.apache.carbondata.store.api; +package org.apache.carbondata.sdk.store; -import java.io.IOException; +import java.util.Iterator; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.core.metadata.datatype.StructType; +import org.apache.carbondata.sdk.store.exception.CarbonException; /** - * Public interface to manage table in CarbonStore + * A Mutator is used to perform insert, update, delete operation on the table */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface MetaStore { +public interface Mutator extends TransactionalOperation { + /** - * Create a Table - * @param table descriptor for create table operation - * @throws IOException if network or disk IO error occurs + * Insert a batch of rows if key is not exist, otherwise update the row + * @param row rows to be upsert + * @param schema schema of the input row (fields without the primary key) + * @throws CarbonException if any error occurs */ - void createTable(TableDescriptor table) throws IOException, StoreException; + void upsert(Iterator row, StructType schema) throws CarbonException; /** - * Drop a Table, and remove all data in it - * @param table table identifier - * @throws IOException if network or disk IO error occurs + * Delete a batch of rows + * @param keys keys to be deleted + * @throws CarbonException if any error occurs */ - void dropTable(TableIdentifier table) throws IOException; - - CarbonTable getTable(TableIdentifier table) throws IOException; + void delete(Iterator keys) throws CarbonException; } diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/exception/StoreException.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/PrimaryKey.java similarity index 74% rename from store/core/src/main/java/org/apache/carbondata/store/api/exception/StoreException.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/PrimaryKey.java index 315a09ba5c3..e18c5aded99 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/exception/StoreException.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/PrimaryKey.java @@ -15,19 +15,12 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.exception; +package org.apache.carbondata.sdk.store; -public class StoreException extends Exception { +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; - public StoreException() { - super(); - } - - public StoreException(String message) { - super(message); - } - - public StoreException(Exception e) { - super(e); - } +@InterfaceAudience.User +@InterfaceStability.Unstable +public class PrimaryKey { } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java new file mode 100644 index 00000000000..4eccb6aa878 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java @@ -0,0 +1,47 @@ +/* + * 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.carbondata.sdk.store; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface ResultBatch { + + /** + * Return true if the result is returned in columnar batch, otherwise is row by row. + * By default, it is columnar batch. + */ + default boolean isColumnar() { + return true; + } + + /** + * Return true if there is more elements in this batch. + */ + boolean hasNext(); + + /** + * Return next item. + * If {@link #isColumnar()} return true, there is only one element in this batch + * which is {@link ColumnarBatch}, otherwise, this batch return row by row, caller + * should call next() until no element left. + */ + T next(); +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Row.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Row.java new file mode 100644 index 00000000000..97d2e14c31f --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Row.java @@ -0,0 +1,27 @@ +/* + * 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.carbondata.sdk.store; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface Row { + Object[] get(); +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java new file mode 100644 index 00000000000..37d376afa01 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java @@ -0,0 +1,40 @@ +/* + * 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.carbondata.sdk.store; + +import java.io.Serializable; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +/** + * An unit for the scanner in Carbon Store + */ +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface ScanUnit extends Serializable { + + /** + * Return the list of preferred location of this ScanUnit. + * The default return value is empty string array, which means this ScanUnit + * has no location preference. + */ + default String[] preferredLocations() { + return new String[0]; + } +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java new file mode 100644 index 00000000000..3844e6c3d8f --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java @@ -0,0 +1,88 @@ +/* + * 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.carbondata.sdk.store; + +import java.util.Iterator; +import java.util.List; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +/** + * A Scanner is used to scan the table + */ +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface Scanner { + + /** + * Scan a Table and return matched rows, using default select option + * see {@link #scan(SelectDescriptor, SelectOption)} for more information + * + * @param select descriptor for select operation + * @return matched rows + * @throws CarbonException if any error occurs + */ + List scan(SelectDescriptor select) throws CarbonException; + + /** + * Scan a Table and return matched rows + * @param select descriptor for select operation, including required column, filter, etc + * @return matched rows + * @throws CarbonException if any error occurs + */ + List scan(SelectDescriptor select, SelectOption option) throws CarbonException; + + /** + * Return an array of ScanUnit which will be the input in + * {@link #scan(ScanUnit, SelectDescriptor, SelectOption)} + * + * Implementation will leverage index to prune using specified filter expression + * + * @param table table identifier + * @param filterExpression expression of filter predicate given by user + * @return unit of scan + * @throws CarbonException if any error occurs + */ + ScanUnit[] prune(TableIdentifier table, Expression filterExpression) throws CarbonException; + + /** + * Perform a scan in a distributed compute framework like Spark, Presto, etc. + * Filter/Projection/Limit operation is pushed down to the scan. + * + * This should be used with {@link #prune(TableIdentifier, Expression)} in a distributed + * compute environment. It enables the framework to do a parallel scan by creating + * multiple {@link ScanUnit} and perform parallel scan in worker, such as Spark executor + * + * The return result is in batch so that the caller can start next level of computation + * before getting all results, such as implementing a `prefetch` execution model. + * + * @param input one scan unit + * @param select parameter for scanning + * @return scan result, the result is returned in batch + * @throws CarbonException if any error occurs + */ + Iterator> scan(ScanUnit input, SelectDescriptor select, + SelectOption option) throws CarbonException; + +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java new file mode 100644 index 00000000000..bafa26ea1c7 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java @@ -0,0 +1,28 @@ +/* + * 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.carbondata.sdk.store; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Unstable +public class SelectOption { + public static final String BATCH_SIZE = "batchSize"; + public static final String ENABLE_ROW_MAJOR = "rowMajor"; +} \ No newline at end of file diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/SqlStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java similarity index 93% rename from store/core/src/main/java/org/apache/carbondata/store/api/SqlStore.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java index 3f52eede0af..d8e4f83061b 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/SqlStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.carbondata.store.api; +package org.apache.carbondata.sdk.store; import java.io.IOException; import java.util.List; import org.apache.carbondata.core.datastore.row.CarbonRow; -public interface SqlStore { +public interface SqlOperation { /** * Executor a SQL statement diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java new file mode 100644 index 00000000000..c55856cad55 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java @@ -0,0 +1,35 @@ +/* + * 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.carbondata.sdk.store; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Unstable +public interface TransactionalOperation { + /** + * commit the transaction when operation succeed + */ + void commit(); + + /** + * close the transaction when operation failed + */ + void close(); +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/conf/StoreConf.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java similarity index 98% rename from store/core/src/main/java/org/apache/carbondata/store/api/conf/StoreConf.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java index 5e4bb4a92ec..3dac9253e78 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/conf/StoreConf.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.conf; +package org.apache.carbondata.sdk.store.conf; import java.io.DataInput; import java.io.DataOutput; @@ -26,7 +26,7 @@ import java.util.Set; import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.store.util.StoreUtil; +import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/LoadDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java similarity index 92% rename from store/core/src/main/java/org/apache/carbondata/store/api/descriptor/LoadDescriptor.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java index c3a4ff7b84b..b05bfb6c616 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/LoadDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java @@ -15,12 +15,17 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.descriptor; +package org.apache.carbondata.sdk.store.descriptor; import java.util.HashMap; import java.util.Map; import java.util.Objects; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Evolving public class LoadDescriptor { private TableIdentifier table; diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/SelectDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/SelectDescriptor.java similarity index 86% rename from store/core/src/main/java/org/apache/carbondata/store/api/descriptor/SelectDescriptor.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/SelectDescriptor.java index c3627a913a8..73e97287911 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/SelectDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/SelectDescriptor.java @@ -15,12 +15,16 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.descriptor; +package org.apache.carbondata.sdk.store.descriptor; import java.util.Objects; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.scan.expression.Expression; +@InterfaceAudience.User +@InterfaceStability.Evolving public class SelectDescriptor { private TableIdentifier table; @@ -41,11 +45,11 @@ public SelectDescriptor(TableIdentifier table, String[] projection, this.limit = limit; } - public TableIdentifier getTable() { + public TableIdentifier getTableIdentifier() { return table; } - public void setTable(TableIdentifier table) { + public void setTableIdentifier(TableIdentifier table) { this.table = table; } @@ -81,7 +85,7 @@ private Builder() { } public Builder table(TableIdentifier tableIdentifier) { - select.setTable(tableIdentifier); + select.setTableIdentifier(tableIdentifier); return this; } diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/TableDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java similarity index 98% rename from store/core/src/main/java/org/apache/carbondata/store/api/descriptor/TableDescriptor.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java index 2d677a8e4b0..21ef8813c90 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/TableDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.descriptor; +package org.apache.carbondata.sdk.store.descriptor; import java.util.ArrayList; import java.util.HashMap; diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/TableIdentifier.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java similarity index 77% rename from store/core/src/main/java/org/apache/carbondata/store/api/descriptor/TableIdentifier.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java index ab8edf8da8a..59bf32bdb03 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/descriptor/TableIdentifier.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java @@ -15,9 +15,16 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.descriptor; +package org.apache.carbondata.sdk.store.descriptor; -public class TableIdentifier { +import java.io.Serializable; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Evolving +public class TableIdentifier implements Serializable { private String tableName; private String databaseName; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/CarbonException.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/CarbonException.java new file mode 100644 index 00000000000..e4e0b10e2d7 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/CarbonException.java @@ -0,0 +1,38 @@ +/* + * 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.carbondata.sdk.store.exception; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Evolving +public class CarbonException extends Exception { + + public CarbonException() { + super(); + } + + public CarbonException(String message) { + super(message); + } + + public CarbonException(Exception e) { + super(e); + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/exception/ExecutionTimeoutException.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/ExecutionTimeoutException.java similarity index 78% rename from store/core/src/main/java/org/apache/carbondata/store/api/exception/ExecutionTimeoutException.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/ExecutionTimeoutException.java index 728837d7a08..3ddd44009b4 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/exception/ExecutionTimeoutException.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/ExecutionTimeoutException.java @@ -15,8 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.exception; +package org.apache.carbondata.sdk.store.exception; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Evolving public class ExecutionTimeoutException extends RuntimeException { } diff --git a/store/core/src/main/java/org/apache/carbondata/store/api/exception/SchedulerException.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/SchedulerException.java similarity index 79% rename from store/core/src/main/java/org/apache/carbondata/store/api/exception/SchedulerException.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/SchedulerException.java index 28b8a503508..8a0309fde73 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/api/exception/SchedulerException.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/exception/SchedulerException.java @@ -15,8 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.store.api.exception; +package org.apache.carbondata.sdk.store.exception; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; + +@InterfaceAudience.User +@InterfaceStability.Evolving public class SchedulerException extends RuntimeException { public SchedulerException(String message) { diff --git a/store/core/src/main/java/org/apache/carbondata/store/util/StoreUtil.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/util/StoreUtil.java similarity index 97% rename from store/core/src/main/java/org/apache/carbondata/store/util/StoreUtil.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/util/StoreUtil.java index 775669fcfcd..ca5740ef18d 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/util/StoreUtil.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/util/StoreUtil.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.util; +package org.apache.carbondata.sdk.store.util; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -36,7 +36,7 @@ import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; -import org.apache.carbondata.store.api.conf.StoreConf; +import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; From 63e2c2cba4221f70c788325277b47b93bbc94de3 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Wed, 1 Aug 2018 02:39:45 +0800 Subject: [PATCH 02/14] correct Horizon --- .../org/apache/spark/sql/CarbonSession.scala | 4 +-- .../store/impl/CarbonStoreBase.java | 10 ++++++ .../store/impl/DistributedCarbonStore.java | 11 ++++-- .../store/impl/LocalCarbonStore.java | 12 +++++-- .../store/DistributedCarbonStoreTest.java | 10 +++--- .../store/LocalCarbonStoreTest.java | 10 +++--- .../rest/controller/HorizonController.java | 34 +++++++++---------- .../rest/model/view/CreateTableRequest.java | 4 +-- .../horizon/rest/model/view/LoadRequest.java | 4 +-- .../carbondata/sdk/store/CarbonStore.java | 25 ++++++++++++-- .../apache/carbondata/sdk/store/Scanner.java | 26 ++------------ ...ectDescriptor.java => ScanDescriptor.java} | 12 +++---- 12 files changed, 92 insertions(+), 70 deletions(-) rename store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/{SelectDescriptor.java => ScanDescriptor.java} (91%) diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala index 3a93c091e23..a7b78defae9 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala @@ -41,7 +41,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants import org.apache.carbondata.core.util.{CarbonProperties, CarbonSessionInfo, ThreadLocalSessionInfo} import org.apache.carbondata.sdk.store.{CarbonStore, CarbonStoreFactory} import org.apache.carbondata.sdk.store.conf.StoreConf -import org.apache.carbondata.sdk.store.descriptor.{SelectDescriptor, TableIdentifier} +import org.apache.carbondata.sdk.store.descriptor.{ScanDescriptor, TableIdentifier} import org.apache.carbondata.store.WorkerManager /** @@ -237,7 +237,7 @@ class CarbonSession(@transient val sc: SparkContext, maxRows: Option[Long] = None, localMaxRows: Option[Long] = None): DataFrame = { val table = relation.relation.asInstanceOf[CarbonDatasourceHadoopRelation].carbonTable - val select = new SelectDescriptor( + val select = new ScanDescriptor( new TableIdentifier(table.getTableName, table.getDatabaseName), columns.map(_.name).toArray, if (expr != null) CarbonFilters.transformExpression(expr) else null, diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java index 05e2b3b10de..86583d1c86a 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java @@ -86,6 +86,16 @@ public void dropTable(TableIdentifier table) throws CarbonException { } } + @Override + public CarbonTable getTable(TableIdentifier table) throws CarbonException { + Objects.requireNonNull(table); + try { + return metaProcessor.getTable(table); + } catch (IOException e) { + throw new CarbonException(e); + } + } + @Override public List listTable() throws CarbonException { throw new UnsupportedOperationException(); diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java index e014ff20eb7..45032c8455b 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java @@ -46,9 +46,10 @@ import org.apache.carbondata.processing.util.CarbonLoaderUtil; import org.apache.carbondata.sdk.store.Loader; import org.apache.carbondata.sdk.store.Scanner; +import org.apache.carbondata.sdk.store.SelectOption; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.sdk.store.exception.ExecutionTimeoutException; import org.apache.carbondata.store.impl.master.Schedulable; @@ -125,7 +126,7 @@ public Loader newLoader(LoadDescriptor load) throws CarbonException { } @Override - public List select(SelectDescriptor select) throws CarbonException { + public List scan(ScanDescriptor select) throws CarbonException { Objects.requireNonNull(select); try { CarbonTable carbonTable = metaProcessor.getTable(select.getTableIdentifier()); @@ -140,6 +141,12 @@ public List select(SelectDescriptor select) throws CarbonException { } } + @Override + public List scan(ScanDescriptor select, SelectOption option) + throws CarbonException { + throw new UnsupportedOperationException(); + } + @Override public Scanner newScanner() throws CarbonException { throw new UnsupportedOperationException(); diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index b858fd4d242..71b083a05b2 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -42,9 +42,10 @@ import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; import org.apache.carbondata.sdk.store.Loader; import org.apache.carbondata.sdk.store.Scanner; +import org.apache.carbondata.sdk.store.SelectOption; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.rpc.model.Scan; @@ -160,7 +161,7 @@ private void loadData(CarbonLoadModel model) throws Exception { } @Override - public List select(SelectDescriptor select) throws CarbonException { + public List scan(ScanDescriptor select) throws CarbonException { Objects.requireNonNull(select); try { CarbonTable table = metaProcessor.getTable(select.getTableIdentifier()); @@ -174,8 +175,13 @@ public List select(SelectDescriptor select) throws CarbonException { } } + @Override + public List scan(ScanDescriptor select, SelectOption option) throws CarbonException { + throw new UnsupportedOperationException(); + } + @Override public Scanner newScanner() throws CarbonException { - return null; + throw new UnsupportedOperationException(); } @Override diff --git a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java index bb62b276d20..d507128803e 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java @@ -28,7 +28,7 @@ import org.apache.carbondata.core.scan.expression.LiteralExpression; import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; @@ -114,17 +114,17 @@ public void testSelect() throws IOException, CarbonException { store.loadData(load); // select row - SelectDescriptor select = SelectDescriptor + ScanDescriptor select = ScanDescriptor .builder() .table(tableIdentifier) .select("intField", "stringField") .limit(5) .create(); - List result = store.select(select); + List result = store.scan(select); Assert.assertEquals(5, result.size()); // select row with filter - SelectDescriptor select2 = SelectDescriptor + ScanDescriptor select2 = ScanDescriptor .builder() .table(tableIdentifier) .select("intField", "stringField") @@ -133,7 +133,7 @@ public void testSelect() throws IOException, CarbonException { new LiteralExpression(11, DataTypes.INT))) .limit(5) .create(); - List result2 = store.select(select2); + List result2 = store.scan(select2); Assert.assertEquals(1, result2.size()); store.dropTable(tableIdentifier); diff --git a/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java index 419c00fac4f..09a326280ef 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java @@ -28,7 +28,7 @@ import org.apache.carbondata.core.scan.expression.LiteralExpression; import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; @@ -106,17 +106,17 @@ public void testWriteAndReadFiles() throws IOException, CarbonException { store.loadData(load); // select row - SelectDescriptor select = SelectDescriptor + ScanDescriptor select = ScanDescriptor .builder() .table(tableIdentifier) .select("intField", "stringField") .limit(5) .create(); - List result = store.select(select); + List result = store.scan(select); Assert.assertEquals(5, result.size()); // select row with filter - SelectDescriptor select2 = SelectDescriptor + ScanDescriptor select2 = ScanDescriptor .builder() .table(tableIdentifier) .select("intField", "stringField") @@ -125,7 +125,7 @@ public void testWriteAndReadFiles() throws IOException, CarbonException { new LiteralExpression(11, DataTypes.INT))) .limit(5) .create(); - List result2 = store.select(select2); + List result2 = store.scan(select2); Assert.assertEquals(1, result2.size()); store.dropTable(tableIdentifier); diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java index a273f54418a..c625ec582f4 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java @@ -36,14 +36,14 @@ import org.apache.carbondata.horizon.rest.model.view.LoadRequest; import org.apache.carbondata.horizon.rest.model.view.SelectRequest; import org.apache.carbondata.horizon.rest.model.view.SelectResponse; -import org.apache.carbondata.store.api.CarbonStore; -import org.apache.carbondata.store.api.CarbonStoreFactory; -import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.SelectDescriptor; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.CarbonStore; +import org.apache.carbondata.sdk.store.CarbonStoreFactory; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.springframework.http.HttpStatus; import org.springframework.http.MediaType; @@ -61,7 +61,7 @@ public class HorizonController { private CarbonStore store; - public HorizonController() throws StoreException { + public HorizonController() throws CarbonException { String storeFile = System.getProperty("carbonstore.conf.file"); StoreConf storeConf = new StoreConf(); try { @@ -77,9 +77,9 @@ public HorizonController() throws StoreException { } } catch (UnknownHostException e) { - throw new StoreException(e); + throw new CarbonException(e); } catch (IOException e) { - throw new StoreException(e); + throw new CarbonException(e); } store = CarbonStoreFactory.getDistributedStore("GlobalStore", storeConf); @@ -92,7 +92,7 @@ public ResponseEntity echo(@RequestParam(name = "name") String name) { @RequestMapping(value = "/table/create", produces = MediaType.APPLICATION_JSON_VALUE) public ResponseEntity createTable( - @RequestBody CreateTableRequest request) throws StoreException, IOException { + @RequestBody CreateTableRequest request) throws CarbonException { RequestValidator.validateTable(request); TableDescriptor tableDescriptor = request.convertToDto(); store.createTable(tableDescriptor); @@ -101,7 +101,7 @@ public ResponseEntity createTable( @RequestMapping(value = "/table/drop", produces = MediaType.APPLICATION_JSON_VALUE) public ResponseEntity dropTable( - @RequestBody DropTableRequest request) throws StoreException, IOException { + @RequestBody DropTableRequest request) throws CarbonException { RequestValidator.validateDrop(request); store.dropTable(new TableIdentifier(request.getTableName(), request.getDatabaseName())); return new ResponseEntity<>(String.valueOf(true), HttpStatus.OK); @@ -109,7 +109,7 @@ public ResponseEntity dropTable( @RequestMapping(value = "/table/load", produces = MediaType.APPLICATION_JSON_VALUE) public ResponseEntity load(@RequestBody LoadRequest request) - throws StoreException, IOException { + throws CarbonException, IOException { RequestValidator.validateLoad(request); LoadDescriptor loadDescriptor = request.convertToDto(); store.loadData(loadDescriptor); @@ -118,15 +118,15 @@ public ResponseEntity load(@RequestBody LoadRequest request) @RequestMapping(value = "/table/select", produces = MediaType.APPLICATION_JSON_VALUE) public ResponseEntity select(@RequestBody SelectRequest request) - throws StoreException, IOException { + throws CarbonException { long start = System.currentTimeMillis(); RequestValidator.validateSelect(request); TableIdentifier table = new TableIdentifier(request.getTableName(), request.getDatabaseName()); CarbonTable carbonTable = store.getTable(table); Expression expression = Parser.parseFilter(request.getFilter(), carbonTable); - SelectDescriptor selectDescriptor = new SelectDescriptor( + ScanDescriptor selectDescriptor = new ScanDescriptor( table, request.getSelect(), expression, request.getLimit()); - List result = store.select(selectDescriptor); + List result = store.scan(selectDescriptor); Iterator iterator = result.iterator(); Object[][] output = new Object[result.size()][]; int i = 0; diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/CreateTableRequest.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/CreateTableRequest.java index cf59f7f4e07..623abeef5bb 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/CreateTableRequest.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/CreateTableRequest.java @@ -24,8 +24,8 @@ import org.apache.carbondata.sdk.file.Field; import org.apache.carbondata.sdk.file.Schema; -import org.apache.carbondata.store.api.descriptor.TableDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; public class CreateTableRequest extends Request { diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/LoadRequest.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/LoadRequest.java index c91f5f5f263..dfe21f6b960 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/LoadRequest.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/LoadRequest.java @@ -20,8 +20,8 @@ import java.util.HashMap; import java.util.Map; -import org.apache.carbondata.store.api.descriptor.LoadDescriptor; -import org.apache.carbondata.store.api.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; public class LoadRequest extends Request { diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java index 399802b8341..580070dd1fb 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -23,8 +23,9 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; @@ -54,6 +55,14 @@ public interface CarbonStore extends Closeable { */ void dropTable(TableIdentifier table) throws CarbonException; + /** + * Return CarbonTable object by specified identifier + * @param table table identifier + * @return CarbonTable object + * @throws CarbonException if any error occurs + */ + CarbonTable getTable(TableIdentifier table) throws CarbonException; + /** * @return all table created * @throws CarbonException if any error occurs @@ -122,13 +131,23 @@ default Mutator newMutator() throws CarbonException { ///// Read Operation ///// //////////////////////////////////////////////////////////////////// + /** + * Scan a Table and return matched rows, using default select option + * see {@link #scan(ScanDescriptor, SelectOption)} for more information + * + * @param select descriptor for select operation + * @return matched rows + * @throws CarbonException if any error occurs + */ + List scan(ScanDescriptor select) throws CarbonException; + /** * Scan a Table and return matched rows - * @param select descriptor for scan operation, including required column, filter, etc + * @param select descriptor for select operation, including required column, filter, etc * @return matched rows * @throws CarbonException if any error occurs */ - List select(SelectDescriptor select) throws CarbonException; + List scan(ScanDescriptor select, SelectOption option) throws CarbonException; /** * @return a new Scanner diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java index 3844e6c3d8f..35bebff4bdb 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java @@ -18,13 +18,11 @@ package org.apache.carbondata.sdk.store; import java.util.Iterator; -import java.util.List; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.sdk.store.descriptor.SelectDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; @@ -35,27 +33,9 @@ @InterfaceStability.Unstable public interface Scanner { - /** - * Scan a Table and return matched rows, using default select option - * see {@link #scan(SelectDescriptor, SelectOption)} for more information - * - * @param select descriptor for select operation - * @return matched rows - * @throws CarbonException if any error occurs - */ - List scan(SelectDescriptor select) throws CarbonException; - - /** - * Scan a Table and return matched rows - * @param select descriptor for select operation, including required column, filter, etc - * @return matched rows - * @throws CarbonException if any error occurs - */ - List scan(SelectDescriptor select, SelectOption option) throws CarbonException; - /** * Return an array of ScanUnit which will be the input in - * {@link #scan(ScanUnit, SelectDescriptor, SelectOption)} + * {@link #scan(ScanUnit, ScanDescriptor, SelectOption)} * * Implementation will leverage index to prune using specified filter expression * @@ -82,7 +62,7 @@ public interface Scanner { * @return scan result, the result is returned in batch * @throws CarbonException if any error occurs */ - Iterator> scan(ScanUnit input, SelectDescriptor select, + Iterator> scan(ScanUnit input, ScanDescriptor select, SelectOption option) throws CarbonException; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/SelectDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java similarity index 91% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/SelectDescriptor.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java index 73e97287911..fa562e1f796 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/SelectDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java @@ -25,17 +25,17 @@ @InterfaceAudience.User @InterfaceStability.Evolving -public class SelectDescriptor { +public class ScanDescriptor { private TableIdentifier table; private String[] projection; private Expression filter; private long limit; - private SelectDescriptor() { + private ScanDescriptor() { } - public SelectDescriptor(TableIdentifier table, String[] projection, + public ScanDescriptor(TableIdentifier table, String[] projection, Expression filter, long limit) { Objects.requireNonNull(table); Objects.requireNonNull(projection); @@ -78,10 +78,10 @@ public void setLimit(long limit) { } public static class Builder { - private SelectDescriptor select; + private ScanDescriptor select; private Builder() { - select = new SelectDescriptor(); + select = new ScanDescriptor(); } public Builder table(TableIdentifier tableIdentifier) { @@ -104,7 +104,7 @@ public Builder limit(long limit) { return this; } - public SelectDescriptor create() { + public ScanDescriptor create() { return select; } } From 7173a3b2a575500a613993e556e3986c6d3b81ec Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Wed, 1 Aug 2018 09:30:30 +0800 Subject: [PATCH 03/14] fix CarbonSession --- .../src/main/scala/org/apache/spark/sql/CarbonSession.scala | 2 +- .../src/main/java/org/apache/carbondata/sdk/store/Loader.java | 3 ++- .../src/main/java/org/apache/carbondata/sdk/store/Scanner.java | 3 ++- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala index a7b78defae9..7b095043875 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala @@ -243,7 +243,7 @@ class CarbonSession(@transient val sc: SparkContext, if (expr != null) CarbonFilters.transformExpression(expr) else null, localMaxRows.getOrElse(Long.MaxValue) ) - val rows = store.select(select).iterator() + val rows = store.scan(select).iterator() val output = new java.util.ArrayList[Row]() val maxRowCount = maxRows.getOrElse(Long.MaxValue) var rowCount = 0 diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java index 0c3e73b7f00..cd8f98c1635 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java @@ -17,6 +17,7 @@ package org.apache.carbondata.sdk.store; +import java.io.Serializable; import java.util.Iterator; import org.apache.carbondata.common.annotations.InterfaceAudience; @@ -29,7 +30,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface Loader extends TransactionalOperation { +public interface Loader extends TransactionalOperation, Serializable { /** * Trigger the load operation * @throws CarbonException if any error occurs diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java index 35bebff4bdb..578ca561e5a 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java @@ -17,6 +17,7 @@ package org.apache.carbondata.sdk.store; +import java.io.Serializable; import java.util.Iterator; import org.apache.carbondata.common.annotations.InterfaceAudience; @@ -31,7 +32,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface Scanner { +public interface Scanner extends Serializable { /** * Return an array of ScanUnit which will be the input in From 2bcaa0ea22ec6c28be8123bf14806e69bc34bb53 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Wed, 1 Aug 2018 11:05:38 +0800 Subject: [PATCH 04/14] add Scanner --- store/core/pom.xml | 4 +- .../carbondata/store/impl/BlockScanUnit.java} | 18 ++-- .../store/impl/CarbonStoreBase.java | 7 +- .../store/impl/DistributedCarbonStore.java | 65 ++++++-------- .../store/impl/LocalCarbonStore.java | 2 +- .../store/impl/RowMajorResultBatch.java} | 37 ++++++-- .../carbondata/store/impl/ScannerImpl.java | 88 +++++++++++++++++++ .../apache/carbondata/sdk/store/ScanUnit.java | 2 +- .../apache/carbondata/sdk/store/Scanner.java | 9 +- 9 files changed, 172 insertions(+), 60 deletions(-) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java => core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java} (65%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java => core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java} (54%) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java diff --git a/store/core/pom.xml b/store/core/pom.xml index 44d5ab1da43..c9e498d1af6 100644 --- a/store/core/pom.xml +++ b/store/core/pom.xml @@ -48,8 +48,8 @@ org.apache.maven.plugins maven-compiler-plugin - 1.7 - 1.7 + 1.8 + 1.8 diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java b/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java similarity index 65% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java index 174d46d7f4a..35acfa32643 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnVector.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java @@ -15,13 +15,19 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.datastore.block.Distributable; +import org.apache.carbondata.sdk.store.ScanUnit; -@InterfaceAudience.User -@InterfaceStability.Unstable -public interface ColumnVector { +public class BlockScanUnit implements ScanUnit { + private Distributable distributable; + public BlockScanUnit(Distributable distributable) { + this.distributable = distributable; + } + + public Distributable getDistributable() { + return distributable; + } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java index 86583d1c86a..3a54f7c94ed 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java @@ -122,16 +122,15 @@ public String getTablePath(String tableName, String databaseName) { * Return a mapping of host address to list of block. * This should be invoked in driver side. */ - public static List pruneBlock(CarbonTable table, String[] columns, - Expression filter) throws IOException { + static List pruneBlock(CarbonTable table, Expression filter) throws IOException { Objects.requireNonNull(table); - Objects.requireNonNull(columns); JobConf jobConf = new JobConf(new Configuration()); Job job = new Job(jobConf); CarbonTableInputFormat format; try { + // We just want to do pruning, so passing empty projection columns format = CarbonInputFormatUtil.createCarbonTableInputFormat( - job, table, columns, filter, null, null, true); + job, table, new String[0], filter, null, null, true); } catch (InvalidConfigurationException e) { throw new IOException(e.getMessage()); } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java index 45032c8455b..0740becdaf0 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java @@ -127,57 +127,50 @@ public Loader newLoader(LoadDescriptor load) throws CarbonException { @Override public List scan(ScanDescriptor select) throws CarbonException { - Objects.requireNonNull(select); - try { - CarbonTable carbonTable = metaProcessor.getTable(select.getTableIdentifier()); - return select( - carbonTable, - select.getProjection(), - select.getFilter(), - select.getLimit(), - select.getLimit()); - } catch (IOException e) { - throw new CarbonException(e); - } + return scan(select, null); } @Override - public List scan(ScanDescriptor select, SelectOption option) + public List scan(ScanDescriptor scanDescriptor, SelectOption option) throws CarbonException { - throw new UnsupportedOperationException(); + Objects.requireNonNull(scanDescriptor); + try { + CarbonTable carbonTable = metaProcessor.getTable(scanDescriptor.getTableIdentifier()); + Objects.requireNonNull(carbonTable); + Objects.requireNonNull(scanDescriptor.getProjection()); + if (scanDescriptor.getLimit() < 0) { + throw new IllegalArgumentException("limit should be positive"); + } + + // prune data and get a mapping of worker hostname to list of blocks, + // then add these blocks to the Scan and fire the RPC call + List blockInfos = pruneBlock(carbonTable, scanDescriptor.getFilter()); + return doScan(carbonTable, scanDescriptor.getProjection(), scanDescriptor.getFilter(), + scanDescriptor.getLimit(), blockInfos); + } catch (IOException e) { + throw new CarbonException(e); + } } @Override public Scanner newScanner() throws CarbonException { - throw new UnsupportedOperationException(); + return new ScannerImpl(this); } /** - * Execute search by firing RPC call to worker, return the result rows + * Execute scan by firing RPC call to worker, return the result rows * * @param table table to search - * @param columns projection column names + * @param requiredColumns projection column names * @param filter filter expression - * @param globalLimit max number of rows required in Master - * @param localLimit max number of rows required in Worker + * @param limit max number of rows required * @return CarbonRow */ - private List select(CarbonTable table, String[] columns, Expression filter, - long globalLimit, long localLimit) throws IOException { - Objects.requireNonNull(table); - Objects.requireNonNull(columns); - if (globalLimit < 0 || localLimit < 0) { - throw new IllegalArgumentException("limit should be positive"); - } - + List doScan(CarbonTable table, String[] requiredColumns, Expression filter, + long limit, List blockInfos) + throws IOException { int queryId = random.nextInt(); - List output = new ArrayList<>(); - - // prune data and get a mapping of worker hostname to list of blocks, - // then add these blocks to the Scan and fire the RPC call - List blockInfos = pruneBlock(table, columns, filter); - Map> nodeBlockMapping = CarbonLoaderUtil.nodeBlockMapping( blockInfos, -1, scheduler.getAllWorkerAddresses(), @@ -189,7 +182,7 @@ private List select(CarbonTable table, String[] columns, Expression f for (Map.Entry> entry : entries) { CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(entry.getValue(), entry.getKey()); Scan scan = - new Scan(queryId, split, table.getTableInfo(), columns, filter, localLimit); + new Scan(queryId, split, table.getTableInfo(), requiredColumns, filter, limit); // Find an Endpoind and send the request to it // This RPC is non-blocking so that we do not need to wait before send to next worker @@ -203,7 +196,7 @@ private List select(CarbonTable table, String[] columns, Expression f for (int i = 0; i < length; i++) { Future future = futures.get(i); Schedulable worker = workers.get(i); - if (rowCount < globalLimit) { + if (rowCount < limit) { // wait for worker QueryResponse response = null; try { @@ -217,7 +210,7 @@ private List select(CarbonTable table, String[] columns, Expression f worker.workload.decrementAndGet(); } LOGGER.info("[QueryId: " + queryId + "] receive search response from worker " + worker); - rowCount += onSuccess(queryId, response, output, globalLimit); + rowCount += onSuccess(queryId, response, output, limit); } } return output; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index 71b083a05b2..c393ade8bbb 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -165,7 +165,7 @@ public List scan(ScanDescriptor select) throws CarbonException { Objects.requireNonNull(select); try { CarbonTable table = metaProcessor.getTable(select.getTableIdentifier()); - List blocks = pruneBlock(table, select.getProjection(), select.getFilter()); + List blocks = pruneBlock(table, select.getFilter()); CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, ""); Scan scan = new Scan(0, split, table.getTableInfo(), select.getProjection(), select.getFilter(), select.getLimit()); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java similarity index 54% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java index 2d42d36dc46..9755983824d 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ColumnarBatch.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java @@ -15,13 +15,36 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.annotations.InterfaceStability; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; -@InterfaceAudience.User -@InterfaceStability.Unstable -public interface ColumnarBatch { - ColumnVector get(int ordinal); +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.sdk.store.ResultBatch; + +public class RowMajorResultBatch implements ResultBatch { + + private Iterator iterator; + + RowMajorResultBatch(List rows) { + Objects.requireNonNull(rows); + this.iterator = rows.iterator(); + } + + @Override + public boolean isColumnar() { + return false; + } + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public CarbonRow next() { + return iterator.next(); + } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java new file mode 100644 index 00000000000..36ee9e57a2f --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java @@ -0,0 +1,88 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.carbondata.core.datastore.block.Distributable; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.sdk.store.ResultBatch; +import org.apache.carbondata.sdk.store.ScanUnit; +import org.apache.carbondata.sdk.store.Scanner; +import org.apache.carbondata.sdk.store.SelectOption; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +public class ScannerImpl implements Scanner { + + private DistributedCarbonStore store; + + ScannerImpl(DistributedCarbonStore store) { + this.store = store; + } + + // TODO: implement using RPC + @Override + public List prune(TableIdentifier table, Expression filterExpression) + throws CarbonException { + Objects.requireNonNull(table); + try { + List l = CarbonStoreBase.pruneBlock(store.getTable(table), filterExpression); + return l.stream() + .map((Function) BlockScanUnit::new) + .collect(Collectors.toList()); + } catch (IOException e) { + throw new CarbonException(e); + } + } + + @Override + public Iterator> scan(ScanUnit input, ScanDescriptor select, + SelectOption option) throws CarbonException { + Objects.requireNonNull(input); + Objects.requireNonNull(select); + + List toBeScan = new ArrayList<>(); + if (input instanceof BlockScanUnit) { + toBeScan.add(((BlockScanUnit) input).getDistributable()); + } else { + throw new CarbonException(input.getClass().getName() + " is not supported"); + } + CarbonTable carbonTable = store.getTable(select.getTableIdentifier()); + try { + List rows = store.doScan( + carbonTable, select.getProjection(), select.getFilter(), select.getLimit(), toBeScan); + return rows.stream() + .map((Function>) carbonRow -> + new RowMajorResultBatch(rows)) + .collect(Collectors.toList()) + .iterator(); + } catch (IOException e) { + throw new CarbonException(e); + } + } +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java index 37d376afa01..317de56ce96 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java @@ -27,7 +27,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface ScanUnit extends Serializable { +public interface ScanUnit extends Serializable { /** * Return the list of preferred location of this ScanUnit. diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java index 578ca561e5a..b4385df7bde 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java @@ -19,9 +19,11 @@ import java.io.Serializable; import java.util.Iterator; +import java.util.List; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; @@ -42,10 +44,11 @@ public interface Scanner extends Serializable { * * @param table table identifier * @param filterExpression expression of filter predicate given by user - * @return unit of scan + * @return list of ScanUnit which should be passed to + * {@link #scan(ScanUnit, ScanDescriptor, SelectOption)} * @throws CarbonException if any error occurs */ - ScanUnit[] prune(TableIdentifier table, Expression filterExpression) throws CarbonException; + List prune(TableIdentifier table, Expression filterExpression) throws CarbonException; /** * Perform a scan in a distributed compute framework like Spark, Presto, etc. @@ -63,7 +66,7 @@ public interface Scanner extends Serializable { * @return scan result, the result is returned in batch * @throws CarbonException if any error occurs */ - Iterator> scan(ScanUnit input, ScanDescriptor select, + Iterator> scan(ScanUnit input, ScanDescriptor select, SelectOption option) throws CarbonException; } From ac8b69d0495e6c7f0b5a2296a2c016e4eda0d2a4 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Wed, 1 Aug 2018 17:40:45 +0800 Subject: [PATCH 05/14] wip --- .../org/apache/carbondata/store/impl/LocalCarbonStore.java | 3 ++- .../java/org/apache/carbondata/store/impl/ScannerImpl.java | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index c393ade8bbb..4aff3c9fea3 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -180,7 +180,8 @@ public List scan(ScanDescriptor select, SelectOption option) throws C throw new UnsupportedOperationException(); } - @Override public Scanner newScanner() throws CarbonException { + @Override + public Scanner newScanner() throws CarbonException { throw new UnsupportedOperationException(); } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java index 36ee9e57a2f..637ccf4af26 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java @@ -37,7 +37,7 @@ import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; -public class ScannerImpl implements Scanner { +class ScannerImpl implements Scanner { private DistributedCarbonStore store; @@ -60,6 +60,7 @@ public List prune(TableIdentifier table, Expression filterExpression) } } + // TODO: implement option @Override public Iterator> scan(ScanUnit input, ScanDescriptor select, SelectOption option) throws CarbonException { From 5a3d0633c2a9abe3d6f5afbaec9c7a26c697806f Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 2 Aug 2018 01:09:52 +0800 Subject: [PATCH 06/14] add PruneService in Master --- .../hadoop/api/CarbonInputFormat.java | 28 +++--- .../hadoop/api/CarbonTableInputFormat.java | 11 +++ .../carbondata/store/impl/BlockScanUnit.java | 30 +++++-- .../store/impl/DistributedCarbonStore.java | 70 ++++++++++++++- .../carbondata/store/impl/ScannerImpl.java | 89 ------------------- .../carbondata/store/impl/master/Master.java | 73 +++++++++++++-- .../store/impl/master/PruneServiceImpl.java | 64 +++++++++++++ .../store/impl/master/Scheduler.java | 14 ++- .../store/impl/rpc/PruneService.java | 32 +++++++ .../store/impl/rpc/ServiceFactory.java | 6 ++ .../store/impl/rpc/model/PruneRequest.java | 53 +++++++++++ .../store/impl/rpc/model/PruneResponse.java | 67 ++++++++++++++ .../apache/carbondata/sdk/store/ScanUnit.java | 3 +- .../carbondata/sdk/store/conf/StoreConf.java | 5 ++ 14 files changed, 422 insertions(+), 123 deletions(-) delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java index 70c530f257a..ce0dc72420a 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonInputFormat.java @@ -185,9 +185,8 @@ public static void setPartitionIdList(Configuration configuration, List /** * It sets unresolved filter expression. * - * @param configuration - * @para DataMapJob dataMapJob = getDataMapJob(job.getConfiguration()); -m filterExpression + * @param configuration Hadoop conf + * @param filterExpression filter expression */ public static void setFilterPredicates(Configuration configuration, Expression filterExpression) { if (filterExpression == null) { @@ -245,6 +244,17 @@ public static String getColumnProjection(Configuration configuration) { return configuration.get(COLUMN_PROJECTION); } + public static String[] getProjectionColumns(Configuration configuration) { + String projectionString = getColumnProjection(configuration); + String[] projectColumns; + if (projectionString != null) { + projectColumns = projectionString.split(","); + } else { + projectColumns = new String[]{}; + } + return projectColumns; + } + public static void setFgDataMapPruning(Configuration configuration, boolean enable) { configuration.set(FGDATAMAP_PRUNING, String.valueOf(enable)); } @@ -353,7 +363,7 @@ public AbsoluteTableIdentifier getAbsoluteTableIdentifier(Configuration configur */ @Override public abstract List getSplits(JobContext job) throws IOException; - protected Expression getFilterPredicates(Configuration configuration) { + public static Expression getFilterPredicates(Configuration configuration) { try { String filterExprString = configuration.get(FILTER_PREDICATE); if (filterExprString == null) { @@ -524,7 +534,7 @@ private List getPrunedBlocklets(JobContext job, CarbonTable ca return prunedBlocklets; } - private List getPrunedFiles4ExternalFormat(JobContext job, + public List getPrunedFiles4ExternalFormat(JobContext job, CarbonTable carbonTable, FilterResolverIntf resolver, List segmentIds) throws IOException { ExplainCollector.addPruningInfo(carbonTable.getTableName()); @@ -664,13 +674,7 @@ public QueryModel createQueryModel(InputSplit inputSplit, TaskAttemptContext tas CarbonTable carbonTable = getOrCreateCarbonTable(configuration); // set projection column in the query model - String projectionString = getColumnProjection(configuration); - String[] projectColumns; - if (projectionString != null) { - projectColumns = projectionString.split(","); - } else { - projectColumns = new String[]{}; - } + String[] projectColumns = getProjectionColumns(configuration); QueryModel queryModel = new QueryModelBuilder(carbonTable) .projectColumns(projectColumns) .filterExpression(getFilterPredicates(configuration)) diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java index 84e36e360b0..267ba0fc5de 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java @@ -71,7 +71,9 @@ import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.lib.input.FileSplit; +import org.apache.hadoop.mapreduce.task.JobContextImpl; /** * InputFormat for reading carbondata files with table level metadata support, @@ -98,6 +100,15 @@ public class CarbonTableInputFormat extends CarbonInputFormat { private CarbonTable carbonTable; private ReadCommittedScope readCommittedScope; + public CarbonTableInputFormat() { + } + + public CarbonTableInputFormat(Configuration conf) throws IOException { + this.carbonTable = getOrCreateCarbonTable(conf); + this.readCommittedScope = getReadCommitted( + new JobContextImpl(conf, new JobID()), carbonTable.getAbsoluteTableIdentifier()); + } + /** * Get the cached CarbonTable or create it by TableInfo in `configuration` */ diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java b/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java index 35acfa32643..771cb147418 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java @@ -17,17 +17,35 @@ package org.apache.carbondata.store.impl; -import org.apache.carbondata.core.datastore.block.Distributable; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.carbondata.hadoop.CarbonInputSplit; import org.apache.carbondata.sdk.store.ScanUnit; public class BlockScanUnit implements ScanUnit { - private Distributable distributable; + private CarbonInputSplit inputSplit; + + public BlockScanUnit() { + } + + public BlockScanUnit(CarbonInputSplit inputSplit) { + this.inputSplit = inputSplit; + } + + public CarbonInputSplit getInputSplit() { + return inputSplit; + } - public BlockScanUnit(Distributable distributable) { - this.distributable = distributable; + @Override + public void write(DataOutput out) throws IOException { + inputSplit.write(out); } - public Distributable getDistributable() { - return distributable; + @Override + public void readFields(DataInput in) throws IOException { + inputSplit = new CarbonInputSplit(); + inputSplit.readFields(in); } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java index 0740becdaf0..d0e646a44a0 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -28,6 +29,8 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; @@ -41,24 +44,32 @@ import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; +import org.apache.carbondata.hadoop.api.CarbonInputFormat; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; import org.apache.carbondata.processing.util.CarbonLoaderUtil; import org.apache.carbondata.sdk.store.Loader; +import org.apache.carbondata.sdk.store.ResultBatch; +import org.apache.carbondata.sdk.store.ScanUnit; import org.apache.carbondata.sdk.store.Scanner; import org.apache.carbondata.sdk.store.SelectOption; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.sdk.store.exception.ExecutionTimeoutException; import org.apache.carbondata.store.impl.master.Schedulable; import org.apache.carbondata.store.impl.master.Scheduler; import org.apache.carbondata.store.impl.rpc.model.BaseResponse; import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; +import org.apache.carbondata.store.impl.rpc.model.PruneRequest; +import org.apache.carbondata.store.impl.rpc.model.PruneResponse; import org.apache.carbondata.store.impl.rpc.model.QueryResponse; import org.apache.carbondata.store.impl.rpc.model.Scan; +import org.apache.hadoop.conf.Configuration; + /** * A CarbonStore that leverage multiple servers via RPC calls (Master and Workers) */ @@ -145,7 +156,7 @@ public List scan(ScanDescriptor scanDescriptor, SelectOption option) // prune data and get a mapping of worker hostname to list of blocks, // then add these blocks to the Scan and fire the RPC call List blockInfos = pruneBlock(carbonTable, scanDescriptor.getFilter()); - return doScan(carbonTable, scanDescriptor.getProjection(), scanDescriptor.getFilter(), + return doRemoteScan(carbonTable, scanDescriptor.getProjection(), scanDescriptor.getFilter(), scanDescriptor.getLimit(), blockInfos); } catch (IOException e) { throw new CarbonException(e); @@ -154,7 +165,58 @@ public List scan(ScanDescriptor scanDescriptor, SelectOption option) @Override public Scanner newScanner() throws CarbonException { - return new ScannerImpl(this); + return new Scanner() { + @Override + public List prune(TableIdentifier table, Expression filterExpression) + throws CarbonException { + try { + return doRemotePrune(table, filterExpression); + } catch (IOException e) { + throw new CarbonException(e); + } + } + + @Override + public Iterator> scan(ScanUnit input, ScanDescriptor select, + SelectOption option) throws CarbonException { + List toBeScan = new ArrayList<>(); + if (input instanceof BlockScanUnit) { + toBeScan.add(((BlockScanUnit) input).getInputSplit()); + } else { + throw new CarbonException(input.getClass().getName() + " is not supported"); + } + CarbonTable carbonTable = getTable(select.getTableIdentifier()); + try { + List rows = doRemoteScan( + carbonTable, select.getProjection(), select.getFilter(), select.getLimit(), toBeScan); + return rows.stream() + .map((Function>) carbonRow -> + new RowMajorResultBatch(rows)) + .collect(Collectors.toList()) + .iterator(); + } catch (IOException e) { + throw new CarbonException(e); + } + } + }; + } + + /** + * Trigger a RPC to Carbon Master to do pruning + * @param table table identifier + * @param filterExpression filter expression + * @return list ScanUnit + * @throws IOException if network or disk IO error + */ + private List doRemotePrune(TableIdentifier table, Expression filterExpression) + throws IOException { + Configuration configuration = new Configuration(); + CarbonInputFormat.setTableName(configuration, table.getTableName()); + CarbonInputFormat.setDatabaseName(configuration, table.getDatabaseName()); + CarbonInputFormat.setFilterPredicates(configuration, filterExpression); + PruneRequest request = new PruneRequest(configuration); + PruneResponse response = scheduler.sendRequest(request); + return response.getScanUnits(); } /** @@ -166,8 +228,8 @@ public Scanner newScanner() throws CarbonException { * @param limit max number of rows required * @return CarbonRow */ - List doScan(CarbonTable table, String[] requiredColumns, Expression filter, - long limit, List blockInfos) + private List doRemoteScan(CarbonTable table, String[] requiredColumns, + Expression filter, long limit, List blockInfos) throws IOException { int queryId = random.nextInt(); List output = new ArrayList<>(); diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java deleted file mode 100644 index 637ccf4af26..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/ScannerImpl.java +++ /dev/null @@ -1,89 +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.carbondata.store.impl; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; -import java.util.function.Function; -import java.util.stream.Collectors; - -import org.apache.carbondata.core.datastore.block.Distributable; -import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.sdk.store.ResultBatch; -import org.apache.carbondata.sdk.store.ScanUnit; -import org.apache.carbondata.sdk.store.Scanner; -import org.apache.carbondata.sdk.store.SelectOption; -import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; -import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; -import org.apache.carbondata.sdk.store.exception.CarbonException; - -class ScannerImpl implements Scanner { - - private DistributedCarbonStore store; - - ScannerImpl(DistributedCarbonStore store) { - this.store = store; - } - - // TODO: implement using RPC - @Override - public List prune(TableIdentifier table, Expression filterExpression) - throws CarbonException { - Objects.requireNonNull(table); - try { - List l = CarbonStoreBase.pruneBlock(store.getTable(table), filterExpression); - return l.stream() - .map((Function) BlockScanUnit::new) - .collect(Collectors.toList()); - } catch (IOException e) { - throw new CarbonException(e); - } - } - - // TODO: implement option - @Override - public Iterator> scan(ScanUnit input, ScanDescriptor select, - SelectOption option) throws CarbonException { - Objects.requireNonNull(input); - Objects.requireNonNull(select); - - List toBeScan = new ArrayList<>(); - if (input instanceof BlockScanUnit) { - toBeScan.add(((BlockScanUnit) input).getDistributable()); - } else { - throw new CarbonException(input.getClass().getName() + " is not supported"); - } - CarbonTable carbonTable = store.getTable(select.getTableIdentifier()); - try { - List rows = store.doScan( - carbonTable, select.getProjection(), select.getFilter(), select.getLimit(), toBeScan); - return rows.stream() - .map((Function>) carbonRow -> - new RowMajorResultBatch(rows)) - .collect(Collectors.toList()) - .iterator(); - } catch (IOException e) { - throw new CarbonException(e); - } - } -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java index f52592375c7..e354edc5006 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java @@ -27,6 +27,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.util.StoreUtil; +import org.apache.carbondata.store.impl.rpc.PruneService; import org.apache.carbondata.store.impl.rpc.RegistryService; import org.apache.carbondata.store.impl.rpc.ServiceFactory; import org.apache.carbondata.store.impl.rpc.StoreService; @@ -37,8 +38,8 @@ import org.apache.hadoop.ipc.RPC; /** - * Master of CarbonSearch. - * It provides a Registry service for worker to register. + * Master of CarbonStore. + * It provides a Registry service and Prune service. */ class Master { @@ -46,10 +47,10 @@ class Master { private static LogService LOGGER = LogServiceFactory.getLogService(Master.class.getName()); - // worker host address map to EndpointRef private StoreConf conf; private Configuration hadoopConf; private RPC.Server registryServer = null; + private RPC.Server pruneServer = null; // mapping of worker IP address to worker instance Map workers = new ConcurrentHashMap<>(); @@ -60,9 +61,9 @@ private Master(StoreConf conf) { } /** - * start service and listen on port passed in constructor + * start registry service and listen on port passed in constructor */ - public void startService() throws IOException { + public void startRegistryService() throws IOException { if (registryServer == null) { BindException exception; @@ -99,11 +100,11 @@ public void startService() throws IOException { } LOGGER.info("registry-service started"); } else { - LOGGER.info("Search mode master has already started"); + LOGGER.info("registry-service has already started"); } } - public void stopService() throws InterruptedException { + public void stopRegistryService() throws InterruptedException { if (registryServer != null) { registryServer.stop(); registryServer.join(); @@ -111,6 +112,58 @@ public void stopService() throws InterruptedException { } } + /** + * start registry service and listen on port passed in constructor + */ + public void startPruneService() throws IOException { + if (pruneServer == null) { + + BindException exception; + // we will try to create service at worse case 100 times + int numTry = 100; + String host = conf.masterHost(); + int port = conf.prunePort(); + LOGGER.info("building prune-service on " + host + ":" + port); + + PruneService pruneService = new PruneServiceImpl(); + do { + try { + pruneServer = new RPC.Builder(hadoopConf) + .setBindAddress(host) + .setPort(port) + .setProtocol(PruneService.class) + .setInstance(pruneService) + .build(); + + pruneServer.start(); + numTry = 0; + exception = null; + } catch (BindException e) { + // port is occupied, increase the port number and try again + exception = e; + LOGGER.error(e, "start prune-service failed"); + port = port + 1; + numTry = numTry - 1; + } + } while (numTry > 0); + if (exception != null) { + // we have tried many times, but still failed to find an available port + throw exception; + } + LOGGER.info("prune-service started"); + } else { + LOGGER.info("prune-service has already started"); + } + } + + public void stopPruneService() throws InterruptedException { + if (pruneServer != null) { + pruneServer.stop(); + pruneServer.join(); + pruneServer = null; + } + } + /** * A new searcher is trying to register, add it to the map and connect to this searcher */ @@ -146,7 +199,7 @@ public static synchronized Master getInstance(StoreConf conf) { return instance; } - public static void main(String[] args) throws InterruptedException { + public static void main(String[] args) throws InterruptedException, IOException { if (args.length != 2) { System.err.println("Usage: Master "); return; @@ -155,7 +208,9 @@ public static void main(String[] args) throws InterruptedException { StoreUtil.initLog4j(args[0]); StoreConf conf = new StoreConf(args[1]); Master master = getInstance(conf); - master.stopService(); + master.startRegistryService(); + master.startPruneService(); + Thread.sleep(Long.MAX_VALUE); } } \ No newline at end of file diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java new file mode 100644 index 00000000000..603a6012768 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java @@ -0,0 +1,64 @@ +/* + * 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.carbondata.store.impl.master; + +import java.io.IOException; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; +import org.apache.carbondata.sdk.store.ScanUnit; +import org.apache.carbondata.store.impl.BlockScanUnit; +import org.apache.carbondata.store.impl.rpc.PruneService; +import org.apache.carbondata.store.impl.rpc.model.PruneRequest; +import org.apache.carbondata.store.impl.rpc.model.PruneResponse; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.ProtocolSignature; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; + +public class PruneServiceImpl implements PruneService { + + @Override + public PruneResponse prune(PruneRequest request) throws IOException { + Configuration hadoopConf = request.getHadoopConf(); + Job job = new Job(hadoopConf); + CarbonTableInputFormat format = new CarbonTableInputFormat(hadoopConf); + List prunedResult = format.getSplits(job); + + List output = prunedResult.stream().map( + (Function) inputSplit -> + new BlockScanUnit((CarbonInputSplit) inputSplit) + ).collect(Collectors.toList()); + return new PruneResponse(output); + } + + @Override + public long getProtocolVersion(String protocol, long clientVersion) throws IOException { + return versionID; + } + + @Override + public ProtocolSignature getProtocolSignature(String protocol, long clientVersion, + int clientMethodsHash) throws IOException { + return null; + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java index 2817e9d151c..a494b313853 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java @@ -33,8 +33,11 @@ import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.exception.SchedulerException; +import org.apache.carbondata.store.impl.rpc.PruneService; import org.apache.carbondata.store.impl.rpc.model.BaseResponse; import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; +import org.apache.carbondata.store.impl.rpc.model.PruneRequest; +import org.apache.carbondata.store.impl.rpc.model.PruneResponse; import org.apache.carbondata.store.impl.rpc.model.QueryResponse; import org.apache.carbondata.store.impl.rpc.model.Scan; import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; @@ -53,7 +56,7 @@ public class Scheduler { public Scheduler(StoreConf storeConf) throws IOException { master = Master.getInstance(storeConf); - master.startService(); + master.startRegistryService(); } /** @@ -79,6 +82,13 @@ public BaseResponse sendRequest(final Schedulable worker, return worker.service.loadData(request); } + private PruneService pruneService; + + public PruneResponse sendRequest(final PruneRequest request) throws IOException { + LOGGER.info("sending prune request to " + pruneService); + return pruneService.prune(request); + } + public Schedulable pickWorker(String splitAddress) { Schedulable worker = master.workers.get(splitAddress); // no local worker available, choose one worker randomly @@ -127,7 +137,7 @@ public void stopAllWorkers() throws IOException { } public void stopService() throws InterruptedException { - master.stopService(); + master.stopRegistryService(); } public List getAllWorkerAddresses() { diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java new file mode 100644 index 00000000000..daef01bc02b --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java @@ -0,0 +1,32 @@ +/* + * 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.carbondata.store.impl.rpc; + +import java.io.IOException; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.store.impl.rpc.model.PruneRequest; +import org.apache.carbondata.store.impl.rpc.model.PruneResponse; + +import org.apache.hadoop.ipc.VersionedProtocol; + +@InterfaceAudience.Internal +public interface PruneService extends VersionedProtocol { + long versionID = 1L; + PruneResponse prune(PruneRequest request) throws IOException; +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java index 852f14fe0a2..ad8f22cd0c3 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java @@ -40,4 +40,10 @@ public static RegistryService createRegistryService(String host, int port) throw return RPC.getProxy( RegistryService.class, RegistryService.versionID, address, new Configuration()); } + + public static PruneService createPruneService(String host, int port) throws IOException { + InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); + return RPC.getProxy( + PruneService.class, PruneService.versionID, address, new Configuration()); + } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java new file mode 100644 index 00000000000..0223c380164 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java @@ -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.carbondata.store.impl.rpc.model; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.Serializable; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Writable; + +public class PruneRequest implements Serializable, Writable { + + private Configuration hadoopConf; + + public PruneRequest() { + } + + public PruneRequest(Configuration hadoopConf) { + this.hadoopConf = hadoopConf; + } + + public Configuration getHadoopConf() { + return hadoopConf; + } + + @Override + public void write(DataOutput out) throws IOException { + hadoopConf.write(out); + } + + @Override + public void readFields(DataInput in) throws IOException { + this.hadoopConf = new Configuration(); + this.hadoopConf.readFields(in); + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java new file mode 100644 index 00000000000..9383ac1595c --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java @@ -0,0 +1,67 @@ +/* + * 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.carbondata.store.impl.rpc.model; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.sdk.store.ScanUnit; +import org.apache.carbondata.store.impl.BlockScanUnit; + +import org.apache.hadoop.io.Writable; + +@InterfaceAudience.Internal +public class PruneResponse implements Serializable, Writable { + + private List scanUnits; + + public PruneResponse() { + } + + public PruneResponse(List scanUnits) { + this.scanUnits = scanUnits; + } + + public List getScanUnits() { + return scanUnits; + } + + @Override + public void write(DataOutput out) throws IOException { + out.writeInt(scanUnits.size()); + for (ScanUnit scanUnit : scanUnits) { + scanUnit.write(out); + } + } + + @Override + public void readFields(DataInput in) throws IOException { + int size = in.readInt(); + scanUnits = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + BlockScanUnit scanUnit = new BlockScanUnit(); + scanUnit.readFields(in); + scanUnits.add(scanUnit); + } + } +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java index 317de56ce96..a5d869cbe69 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java @@ -21,13 +21,14 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.metadata.schema.table.Writable; /** * An unit for the scanner in Carbon Store */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface ScanUnit extends Serializable { +public interface ScanUnit extends Serializable, Writable { /** * Return the list of preferred location of this ScanUnit. diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java index 3dac9253e78..89879a7b06b 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java @@ -45,6 +45,7 @@ public class StoreConf implements Serializable, Writable { public static final String WORKER_CORE_NUM = "carbon.worker.core.num"; public static final String MASTER_HOST = "carbon.master.host"; public static final String MASTER_PORT = "carbon.master.port"; + public static final String PRUNE_PORT = "carbon.master.prune.port"; public static final String STORE_TEMP_LOCATION = "carbon.store.temp.location"; public static final String STORE_LOCATION = "carbon.store.location"; @@ -112,6 +113,10 @@ public int masterPort() { return intValue(MASTER_PORT); } + public int prunePort() { + return intValue(PRUNE_PORT); + } + public String workerHost() { return stringValue(WORKER_HOST); } From ef0aa9a1c54fe6be0cc29c2764b279ea0541d8f0 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Fri, 3 Aug 2018 03:28:45 +0800 Subject: [PATCH 07/14] support CarbonStore client --- .../core/datastore/row/CarbonRow.java | 30 +- .../carbondata/hadoop/CarbonInputSplit.java | 12 + .../org/apache/spark/sql/CarbonSession.scala | 4 +- store/conf/store.conf | 4 +- .../store/impl/CarbonStoreBase.java | 199 ----------- .../store/impl/DistributedCarbonStore.java | 317 ------------------ .../store/impl/LocalCarbonStore.java | 34 +- .../{MetaProcessor.java => TableManager.java} | 169 +++++++++- .../carbondata/store/impl/master/Master.java | 102 ++++-- .../store/impl/master/PruneServiceImpl.java | 31 +- .../store/impl/master/Scheduler.java | 79 +---- .../store/impl/master/StoreServiceImpl.java | 100 ++++++ .../store/impl/rpc/ServiceFactory.java | 11 - .../store/impl/rpc/StoreService.java | 40 --- ...stHandler.java => DataRequestHandler.java} | 30 +- ...eServiceImpl.java => DataServiceImpl.java} | 36 +- .../carbondata/store/impl/worker/Worker.java | 14 +- ...reTest.java => RemoteCarbonStoreTest.java} | 33 +- .../carbondata/sdk/file/AvroCarbonWriter.java | 50 +-- .../sdk/file/CarbonWriterBuilder.java | 4 +- .../org/apache/carbondata/sdk/file/Field.java | 15 +- .../apache/carbondata/sdk/file/Schema.java | 3 +- .../carbondata/sdk/store}/BlockScanUnit.java | 25 +- .../carbondata/sdk/store/CarbonStore.java | 10 +- .../sdk/store/CarbonStoreFactory.java | 25 +- .../carbondata/sdk/store/DataServicePool.java | 44 +++ .../sdk/store/RemoteCarbonStore.java | 130 +++++++ .../sdk/store}/RowMajorResultBatch.java | 3 +- .../apache/carbondata/sdk/store/Scanner.java | 2 +- .../carbondata/sdk/store/ScannerImpl.java | 122 +++++++ .../carbondata/sdk/store}/Schedulable.java | 9 +- .../carbondata/sdk/store/conf/StoreConf.java | 13 +- .../sdk/store/descriptor/LoadDescriptor.java | 32 +- .../sdk/store/descriptor/ScanDescriptor.java | 37 +- .../sdk/store/descriptor/TableDescriptor.java | 44 ++- .../sdk/store/descriptor/TableIdentifier.java | 21 +- .../sdk/store/service/DataService.java | 47 +++ .../sdk/store/service}/PruneService.java | 6 +- .../sdk/store/service/ServiceFactory.java | 56 ++++ .../sdk/store/service/StoreService.java | 53 +++ .../store/service}/model/BaseResponse.java | 2 +- .../store/service}/model/LoadDataRequest.java | 2 +- .../store/service}/model/PruneRequest.java | 2 +- .../store/service}/model/PruneResponse.java | 4 +- .../sdk/store/service/model/ScanRequest.java} | 8 +- .../store/service/model/ScanResponse.java} | 8 +- 46 files changed, 1164 insertions(+), 858 deletions(-) delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java rename store/core/src/main/java/org/apache/carbondata/store/impl/{MetaProcessor.java => TableManager.java} (51%) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/rpc/StoreService.java rename store/core/src/main/java/org/apache/carbondata/store/impl/worker/{RequestHandler.java => DataRequestHandler.java} (86%) rename store/core/src/main/java/org/apache/carbondata/store/impl/worker/{StoreServiceImpl.java => DataServiceImpl.java} (60%) rename store/core/src/test/java/org/apache/carbondata/store/{DistributedCarbonStoreTest.java => RemoteCarbonStoreTest.java} (87%) rename store/{core/src/main/java/org/apache/carbondata/store/impl => sdk/src/main/java/org/apache/carbondata/sdk/store}/BlockScanUnit.java (69%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java rename store/{core/src/main/java/org/apache/carbondata/store/impl => sdk/src/main/java/org/apache/carbondata/sdk/store}/RowMajorResultBatch.java (93%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java rename store/{core/src/main/java/org/apache/carbondata/store/impl/master => sdk/src/main/java/org/apache/carbondata/sdk/store}/Schedulable.java (89%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc => sdk/src/main/java/org/apache/carbondata/sdk/store/service}/PruneService.java (85%) create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java create mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc => sdk/src/main/java/org/apache/carbondata/sdk/store/service}/model/BaseResponse.java (97%) rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc => sdk/src/main/java/org/apache/carbondata/sdk/store/service}/model/LoadDataRequest.java (97%) rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc => sdk/src/main/java/org/apache/carbondata/sdk/store/service}/model/PruneRequest.java (96%) rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc => sdk/src/main/java/org/apache/carbondata/sdk/store/service}/model/PruneResponse.java (94%) rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc/model/Scan.java => sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java} (93%) rename store/{core/src/main/java/org/apache/carbondata/store/impl/rpc/model/QueryResponse.java => sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java} (88%) diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java b/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java index 48775d4bf8d..1f1f0878a55 100644 --- a/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java +++ b/core/src/main/java/org/apache/carbondata/core/datastore/row/CarbonRow.java @@ -17,13 +17,21 @@ package org.apache.carbondata.core.datastore.row; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.io.Serializable; import java.util.Arrays; +import org.apache.carbondata.core.metadata.schema.table.Writable; +import org.apache.carbondata.core.util.ObjectSerializationUtil; + +import org.apache.hadoop.io.WritableUtils; + /** * This row class is used to transfer the row data from one step to other step */ -public class CarbonRow implements Serializable { +public class CarbonRow implements Serializable, Writable { private Object[] data; @@ -87,4 +95,24 @@ public void setRangeId(short rangeId) { public void clearData() { this.data = null; } + + @Override + public void write(DataOutput out) throws IOException { + WritableUtils.writeCompressedByteArray(out, ObjectSerializationUtil.serialize(data)); + WritableUtils.writeCompressedByteArray(out, ObjectSerializationUtil.serialize(rawData)); + out.writeShort(rangeId); + } + + @Override + public void readFields(DataInput in) throws IOException { + try { + data = (Object[]) ObjectSerializationUtil.deserialize( + WritableUtils.readCompressedByteArray(in)); + rawData = (Object[]) ObjectSerializationUtil.deserialize( + WritableUtils.readCompressedByteArray(in)); + } catch (ClassNotFoundException e) { + throw new IOException(e); + } + rangeId = in.readShort(); + } } diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java index 405ff532afa..f59f72db7b3 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java @@ -444,4 +444,16 @@ public void setFormat(FileFormat fileFormat) { public Blocklet makeBlocklet() { return new Blocklet(getPath().getName(), blockletId); } + + public String[] preferredLocations() { + if (CarbonProperties.isTaskLocality()) { + try { + return getLocations(); + } catch (IOException e) { + return new String[0]; + } + } else { + return new String[0]; + } + } } diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala index 7b095043875..a1687e8369a 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala @@ -200,12 +200,12 @@ class CarbonSession(@transient val sc: SparkContext, val storeConf = new StoreConf() storeConf.conf(StoreConf.STORE_LOCATION, CarbonProperties.getStorePath) storeConf.conf(StoreConf.MASTER_HOST, InetAddress.getLocalHost.getHostAddress) - storeConf.conf(StoreConf.MASTER_PORT, CarbonProperties.getSearchMasterPort) + storeConf.conf(StoreConf.REGISTRY_PORT, CarbonProperties.getSearchMasterPort) storeConf.conf(StoreConf.WORKER_HOST, InetAddress.getLocalHost.getHostAddress) storeConf.conf(StoreConf.WORKER_PORT, CarbonProperties.getSearchWorkerPort) storeConf.conf(StoreConf.WORKER_CORE_NUM, 2) - store = CarbonStoreFactory.getDistributedStore("GlobalStore", storeConf) + store = CarbonStoreFactory.getRemoteStore("GlobalStore", storeConf) CarbonProperties.enableSearchMode(true) CarbonProperties.getInstance() .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "false") diff --git a/store/conf/store.conf b/store/conf/store.conf index 7f180764a00..9902061f285 100644 --- a/store/conf/store.conf +++ b/store/conf/store.conf @@ -6,5 +6,7 @@ carbon.store.temp.location=/tmp/carbon.store.temp # worker and master carbon.master.host=127.0.0.1 -carbon.master.port=10020 +carbon.master.registry.port=10020 +carbon.master.prune.port=10120 +carbon.master.store.port=9020 carbon.store.location=/tmp/carbon.store \ No newline at end of file diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java b/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java deleted file mode 100644 index 3a54f7c94ed..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/CarbonStoreBase.java +++ /dev/null @@ -1,199 +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.carbondata.store.impl; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; -import java.util.Objects; - -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.block.Distributable; -import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.exception.InvalidConfigurationException; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.model.QueryModelBuilder; -import org.apache.carbondata.core.util.CarbonTaskInfo; -import org.apache.carbondata.core.util.ThreadLocalTaskInfo; -import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; -import org.apache.carbondata.hadoop.CarbonRecordReader; -import org.apache.carbondata.hadoop.api.CarbonInputFormat; -import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; -import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; -import org.apache.carbondata.sdk.store.CarbonStore; -import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; -import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; -import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.store.impl.rpc.model.Scan; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; - -/** - * Provides base functionality of CarbonStore, it contains basic implementation of metadata - * management, data pruning and data scan logic. - */ -@InterfaceAudience.Internal -public abstract class CarbonStoreBase implements CarbonStore { - - private static LogService LOGGER = - LogServiceFactory.getLogService(CarbonStoreBase.class.getCanonicalName()); - - MetaProcessor metaProcessor; - private StoreConf storeConf; - - CarbonStoreBase(StoreConf storeConf) { - this.storeConf = storeConf; - this.metaProcessor = new MetaProcessor(this); - } - - @Override - public void createTable(TableDescriptor descriptor) throws CarbonException { - Objects.requireNonNull(descriptor); - metaProcessor.createTable(descriptor); - } - - @Override - public void dropTable(TableIdentifier table) throws CarbonException { - Objects.requireNonNull(table); - try { - metaProcessor.dropTable(table); - } catch (IOException e) { - throw new CarbonException(e); - } - } - - @Override - public CarbonTable getTable(TableIdentifier table) throws CarbonException { - Objects.requireNonNull(table); - try { - return metaProcessor.getTable(table); - } catch (IOException e) { - throw new CarbonException(e); - } - } - - @Override - public List listTable() throws CarbonException { - throw new UnsupportedOperationException(); - } - - @Override - public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTable(TableIdentifier table, TableDescriptor newTable) throws CarbonException { - throw new UnsupportedOperationException(); - } - - public String getTablePath(String tableName, String databaseName) { - Objects.requireNonNull(tableName); - Objects.requireNonNull(databaseName); - return String.format("%s/%s", storeConf.storeLocation(), tableName); - } - - /** - * Prune data by using CarbonInputFormat.getSplit - * Return a mapping of host address to list of block. - * This should be invoked in driver side. - */ - static List pruneBlock(CarbonTable table, Expression filter) throws IOException { - Objects.requireNonNull(table); - JobConf jobConf = new JobConf(new Configuration()); - Job job = new Job(jobConf); - CarbonTableInputFormat format; - try { - // We just want to do pruning, so passing empty projection columns - format = CarbonInputFormatUtil.createCarbonTableInputFormat( - job, table, new String[0], filter, null, null, true); - } catch (InvalidConfigurationException e) { - throw new IOException(e.getMessage()); - } - - // We will do FG pruning in reader side, so don't do it here - CarbonInputFormat.setFgDataMapPruning(job.getConfiguration(), false); - List splits = format.getSplits(job); - List blockInfos = new ArrayList<>(splits.size()); - for (InputSplit split : splits) { - blockInfos.add((Distributable) split); - } - return blockInfos; - } - - /** - * Scan data and return matched rows. This should be invoked in worker side. - * @param table carbon table - * @param scan scan parameter - * @return matched rows - * @throws IOException if IO error occurs - */ - public static List scan(CarbonTable table, Scan scan) throws IOException { - CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo(); - carbonTaskInfo.setTaskId(System.nanoTime()); - ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo); - - CarbonMultiBlockSplit mbSplit = scan.getSplit(); - long limit = scan.getLimit(); - QueryModel queryModel = createQueryModel(table, scan); - - LOGGER.info(String.format("[QueryId:%d] %s, number of block: %d", scan.getRequestId(), - queryModel.toString(), mbSplit.getAllSplits().size())); - - // read all rows by the reader - List rows = new LinkedList<>(); - try (CarbonRecordReader reader = new IndexedRecordReader(scan.getRequestId(), - table, queryModel)) { - reader.initialize(mbSplit, null); - - // loop to read required number of rows. - // By default, if user does not specify the limit value, limit is Long.MaxValue - long rowCount = 0; - while (reader.nextKeyValue() && rowCount < limit) { - rows.add(reader.getCurrentValue()); - rowCount++; - } - } catch (InterruptedException e) { - throw new IOException(e); - } - LOGGER.info(String.format("[QueryId:%d] scan completed, return %d rows", - scan.getRequestId(), rows.size())); - return rows; - } - - private static QueryModel createQueryModel(CarbonTable table, Scan scan) { - String[] projectColumns = scan.getProjectColumns(); - Expression filter = null; - if (scan.getFilterExpression() != null) { - filter = scan.getFilterExpression(); - } - return new QueryModelBuilder(table) - .projectColumns(projectColumns) - .filterExpression(filter) - .build(); - } -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java deleted file mode 100644 index d0e646a44a0..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/DistributedCarbonStore.java +++ /dev/null @@ -1,317 +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.carbondata.store.impl; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Random; -import java.util.Set; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; -import java.util.function.Function; -import java.util.stream.Collectors; - -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.block.Distributable; -import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.mutate.CarbonUpdateUtil; -import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; -import org.apache.carbondata.hadoop.api.CarbonInputFormat; -import org.apache.carbondata.processing.loading.model.CarbonLoadModel; -import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; -import org.apache.carbondata.processing.util.CarbonLoaderUtil; -import org.apache.carbondata.sdk.store.Loader; -import org.apache.carbondata.sdk.store.ResultBatch; -import org.apache.carbondata.sdk.store.ScanUnit; -import org.apache.carbondata.sdk.store.Scanner; -import org.apache.carbondata.sdk.store.SelectOption; -import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; -import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; -import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.sdk.store.exception.ExecutionTimeoutException; -import org.apache.carbondata.store.impl.master.Schedulable; -import org.apache.carbondata.store.impl.master.Scheduler; -import org.apache.carbondata.store.impl.rpc.model.BaseResponse; -import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; -import org.apache.carbondata.store.impl.rpc.model.PruneRequest; -import org.apache.carbondata.store.impl.rpc.model.PruneResponse; -import org.apache.carbondata.store.impl.rpc.model.QueryResponse; -import org.apache.carbondata.store.impl.rpc.model.Scan; - -import org.apache.hadoop.conf.Configuration; - -/** - * A CarbonStore that leverage multiple servers via RPC calls (Master and Workers) - */ -@InterfaceAudience.User -@InterfaceStability.Unstable -class DistributedCarbonStore extends CarbonStoreBase { - private static LogService LOGGER = - LogServiceFactory.getLogService(DistributedCarbonStore.class.getCanonicalName()); - private SegmentTxnManager txnManager; - private Scheduler scheduler; - private Random random = new Random(); - - DistributedCarbonStore(StoreConf storeConf) throws IOException { - super(storeConf); - this.scheduler = new Scheduler(storeConf); - txnManager = SegmentTxnManager.getInstance(); - } - - @Override - public void loadData(LoadDescriptor load) throws CarbonException { - Objects.requireNonNull(load); - CarbonTable table = null; - try { - table = metaProcessor.getTable(load.getTable()); - } catch (IOException e) { - throw new CarbonException(e); - } - CarbonLoadModelBuilder builder = new CarbonLoadModelBuilder(table); - builder.setInputPath(load.getInputPath()); - CarbonLoadModel loadModel; - try { - loadModel = builder.build(load.getOptions(), System.currentTimeMillis(), "0"); - } catch (InvalidLoadOptionException e) { - LOGGER.error(e, "Invalid loadDescriptor options"); - throw new CarbonException(e); - } catch (IOException e) { - LOGGER.error(e, "Failed to loadDescriptor data"); - throw new CarbonException(e); - } - - Schedulable worker = scheduler.pickNexWorker(); - try { - if (loadModel.getFactTimeStamp() == 0) { - loadModel.setFactTimeStamp(CarbonUpdateUtil.readCurrentTime()); - } - txnManager.openSegment(loadModel, load.isOverwrite()); - LoadDataRequest request = new LoadDataRequest(loadModel); - BaseResponse response = scheduler.sendRequest(worker, request); - if (Status.SUCCESS.ordinal() == response.getStatus()) { - txnManager.commitSegment(loadModel); - } else { - txnManager.closeSegment(loadModel); - throw new CarbonException(response.getMessage()); - } - } catch (IOException e) { - throw new CarbonException(e); - } finally { - worker.workload.decrementAndGet(); - } - } - - @Override - public Loader newLoader(LoadDescriptor load) throws CarbonException { - throw new UnsupportedOperationException(); - } - - @Override - public List scan(ScanDescriptor select) throws CarbonException { - return scan(select, null); - } - - @Override - public List scan(ScanDescriptor scanDescriptor, SelectOption option) - throws CarbonException { - Objects.requireNonNull(scanDescriptor); - try { - CarbonTable carbonTable = metaProcessor.getTable(scanDescriptor.getTableIdentifier()); - Objects.requireNonNull(carbonTable); - Objects.requireNonNull(scanDescriptor.getProjection()); - if (scanDescriptor.getLimit() < 0) { - throw new IllegalArgumentException("limit should be positive"); - } - - // prune data and get a mapping of worker hostname to list of blocks, - // then add these blocks to the Scan and fire the RPC call - List blockInfos = pruneBlock(carbonTable, scanDescriptor.getFilter()); - return doRemoteScan(carbonTable, scanDescriptor.getProjection(), scanDescriptor.getFilter(), - scanDescriptor.getLimit(), blockInfos); - } catch (IOException e) { - throw new CarbonException(e); - } - } - - @Override - public Scanner newScanner() throws CarbonException { - return new Scanner() { - @Override - public List prune(TableIdentifier table, Expression filterExpression) - throws CarbonException { - try { - return doRemotePrune(table, filterExpression); - } catch (IOException e) { - throw new CarbonException(e); - } - } - - @Override - public Iterator> scan(ScanUnit input, ScanDescriptor select, - SelectOption option) throws CarbonException { - List toBeScan = new ArrayList<>(); - if (input instanceof BlockScanUnit) { - toBeScan.add(((BlockScanUnit) input).getInputSplit()); - } else { - throw new CarbonException(input.getClass().getName() + " is not supported"); - } - CarbonTable carbonTable = getTable(select.getTableIdentifier()); - try { - List rows = doRemoteScan( - carbonTable, select.getProjection(), select.getFilter(), select.getLimit(), toBeScan); - return rows.stream() - .map((Function>) carbonRow -> - new RowMajorResultBatch(rows)) - .collect(Collectors.toList()) - .iterator(); - } catch (IOException e) { - throw new CarbonException(e); - } - } - }; - } - - /** - * Trigger a RPC to Carbon Master to do pruning - * @param table table identifier - * @param filterExpression filter expression - * @return list ScanUnit - * @throws IOException if network or disk IO error - */ - private List doRemotePrune(TableIdentifier table, Expression filterExpression) - throws IOException { - Configuration configuration = new Configuration(); - CarbonInputFormat.setTableName(configuration, table.getTableName()); - CarbonInputFormat.setDatabaseName(configuration, table.getDatabaseName()); - CarbonInputFormat.setFilterPredicates(configuration, filterExpression); - PruneRequest request = new PruneRequest(configuration); - PruneResponse response = scheduler.sendRequest(request); - return response.getScanUnits(); - } - - /** - * Execute scan by firing RPC call to worker, return the result rows - * - * @param table table to search - * @param requiredColumns projection column names - * @param filter filter expression - * @param limit max number of rows required - * @return CarbonRow - */ - private List doRemoteScan(CarbonTable table, String[] requiredColumns, - Expression filter, long limit, List blockInfos) - throws IOException { - int queryId = random.nextInt(); - List output = new ArrayList<>(); - Map> nodeBlockMapping = - CarbonLoaderUtil.nodeBlockMapping( - blockInfos, -1, scheduler.getAllWorkerAddresses(), - CarbonLoaderUtil.BlockAssignmentStrategy.BLOCK_NUM_FIRST, null); - - Set>> entries = nodeBlockMapping.entrySet(); - List> futures = new ArrayList<>(entries.size()); - List workers = new ArrayList<>(entries.size()); - for (Map.Entry> entry : entries) { - CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(entry.getValue(), entry.getKey()); - Scan scan = - new Scan(queryId, split, table.getTableInfo(), requiredColumns, filter, limit); - - // Find an Endpoind and send the request to it - // This RPC is non-blocking so that we do not need to wait before send to next worker - Schedulable worker = scheduler.pickWorker(entry.getKey()); - workers.add(worker); - futures.add(scheduler.sendRequestAsync(worker, scan)); - } - - int rowCount = 0; - int length = futures.size(); - for (int i = 0; i < length; i++) { - Future future = futures.get(i); - Schedulable worker = workers.get(i); - if (rowCount < limit) { - // wait for worker - QueryResponse response = null; - try { - response = future - .get((long) (CarbonProperties.getInstance().getQueryTimeout()), TimeUnit.SECONDS); - } catch (ExecutionException | InterruptedException e) { - throw new IOException("exception in worker: " + e.getMessage()); - } catch (TimeoutException t) { - throw new ExecutionTimeoutException(); - } finally { - worker.workload.decrementAndGet(); - } - LOGGER.info("[QueryId: " + queryId + "] receive search response from worker " + worker); - rowCount += onSuccess(queryId, response, output, limit); - } - } - return output; - } - - private int onSuccess(int queryId, QueryResponse result, List output, long globalLimit) - throws IOException { - // in case of RPC success, collect all rows in response message - if (result.getQueryId() != queryId) { - throw new IOException( - "queryId in response does not match request: " + result.getQueryId() + " != " + queryId); - } - if (result.getStatus() != Status.SUCCESS.ordinal()) { - throw new IOException("failure in worker: " + result.getMessage()); - } - int rowCount = 0; - Object[][] rows = result.getRows(); - for (Object[] row : rows) { - output.add(new CarbonRow(row)); - rowCount++; - if (rowCount >= globalLimit) { - break; - } - } - LOGGER.info("[QueryId:" + queryId + "] accumulated result size " + rowCount); - return rowCount; - } - - @Override - public void close() throws IOException { - LOGGER.info("Shutting down all workers..."); - scheduler.stopAllWorkers(); - LOGGER.info("All workers are shut down"); - try { - LOGGER.info("Stopping master..."); - scheduler.stopService(); - LOGGER.info("Master stopped"); - } catch (InterruptedException e) { - throw new IOException(e); - } - } -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index 4aff3c9fea3..c825bea58ad 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -40,15 +40,16 @@ import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; +import org.apache.carbondata.sdk.store.CarbonStore; import org.apache.carbondata.sdk.store.Loader; import org.apache.carbondata.sdk.store.Scanner; -import org.apache.carbondata.sdk.store.SelectOption; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.service.model.ScanRequest; import org.apache.carbondata.sdk.store.util.StoreUtil; -import org.apache.carbondata.store.impl.rpc.model.Scan; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; @@ -66,7 +67,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -class LocalCarbonStore extends CarbonStoreBase { +public class LocalCarbonStore extends TableManager implements CarbonStore { private static final LogService LOGGER = LogServiceFactory.getLogService(LocalCarbonStore.class.getName()); @@ -75,11 +76,11 @@ class LocalCarbonStore extends CarbonStoreBase { private Configuration configuration; private SegmentTxnManager txnManager; - LocalCarbonStore(StoreConf storeConf) { + public LocalCarbonStore(StoreConf storeConf) { this(storeConf, new Configuration()); } - LocalCarbonStore(StoreConf storeConf, Configuration hadoopConf) { + public LocalCarbonStore(StoreConf storeConf, Configuration hadoopConf) { super(storeConf); this.storeConf = storeConf; this.txnManager = SegmentTxnManager.getInstance(); @@ -91,7 +92,7 @@ public void loadData(LoadDescriptor load) throws CarbonException { Objects.requireNonNull(load); CarbonLoadModel loadModel; try { - CarbonTable table = metaProcessor.getTable(load.getTable()); + CarbonTable table = getTable(load.getTable()); CarbonLoadModelBuilder modelBuilder = new CarbonLoadModelBuilder(table); modelBuilder.setInputPath(load.getInputPath()); loadModel = modelBuilder.build(load.getOptions(), System.currentTimeMillis(), "0"); @@ -161,14 +162,15 @@ private void loadData(CarbonLoadModel model) throws Exception { } @Override - public List scan(ScanDescriptor select) throws CarbonException { - Objects.requireNonNull(select); + public List scan(ScanDescriptor scanDescriptor) throws CarbonException { + Objects.requireNonNull(scanDescriptor); try { - CarbonTable table = metaProcessor.getTable(select.getTableIdentifier()); - List blocks = pruneBlock(table, select.getFilter()); + CarbonTable table = getTable(scanDescriptor.getTableIdentifier()); + List blocks = pruneBlock(table, scanDescriptor.getFilter()); CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, ""); - Scan scan = new Scan(0, split, table.getTableInfo(), select.getProjection(), - select.getFilter(), select.getLimit()); + ScanRequest scan = + new ScanRequest(0, split, table.getTableInfo(), scanDescriptor.getProjection(), + scanDescriptor.getFilter(), scanDescriptor.getLimit()); return scan(table, scan); } catch (IOException e) { throw new CarbonException(e); @@ -176,17 +178,11 @@ public List scan(ScanDescriptor select) throws CarbonException { } @Override - public List scan(ScanDescriptor select, SelectOption option) throws CarbonException { - throw new UnsupportedOperationException(); - } - - @Override - public Scanner newScanner() throws CarbonException { + public Scanner newScanner(TableIdentifier tableIdentifier) throws CarbonException { throw new UnsupportedOperationException(); } @Override public void close() throws IOException { - } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java b/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java similarity index 51% rename from store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java index a2dd95525b5..142624c0850 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/MetaProcessor.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java @@ -18,15 +18,22 @@ package org.apache.carbondata.store.impl; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Objects; +import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.datastore.block.Distributable; import org.apache.carbondata.core.datastore.impl.FileFactory; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.exception.InvalidConfigurationException; import org.apache.carbondata.core.fileoperations.FileWriteOperation; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.metadata.converter.SchemaConverter; @@ -37,31 +44,51 @@ import org.apache.carbondata.core.metadata.schema.table.TableSchema; import org.apache.carbondata.core.metadata.schema.table.TableSchemaBuilder; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.core.scan.model.QueryModel; +import org.apache.carbondata.core.scan.model.QueryModelBuilder; +import org.apache.carbondata.core.util.CarbonTaskInfo; import org.apache.carbondata.core.util.CarbonUtil; +import org.apache.carbondata.core.util.ThreadLocalTaskInfo; import org.apache.carbondata.core.util.path.CarbonTablePath; import org.apache.carbondata.core.writer.ThriftWriter; +import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; +import org.apache.carbondata.hadoop.CarbonRecordReader; +import org.apache.carbondata.hadoop.api.CarbonInputFormat; +import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; +import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; import org.apache.carbondata.sdk.file.CarbonWriterBuilder; import org.apache.carbondata.sdk.file.Field; +import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.service.model.ScanRequest; -class MetaProcessor { +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; + +/** + * Provides table management. + */ +@InterfaceAudience.Internal +public class TableManager { private static LogService LOGGER = - LogServiceFactory.getLogService(MetaProcessor.class.getCanonicalName()); + LogServiceFactory.getLogService(TableManager.class.getCanonicalName()); - private CarbonStoreBase store; - - MetaProcessor(CarbonStoreBase store) { - this.store = store; - } + private StoreConf storeConf; // mapping of table path to CarbonTable object private Map cache = new HashMap<>(); - public void createTable(TableDescriptor descriptor) throws - CarbonException { + public TableManager(StoreConf storeConf) { + this.storeConf = storeConf; + } + + public void createTable(TableDescriptor descriptor) throws CarbonException { TableIdentifier table = descriptor.getTable(); Field[] fields = descriptor.getSchema().getFields(); // sort_columns @@ -88,7 +115,7 @@ public void createTable(TableDescriptor descriptor) throws String tablePath = descriptor.getTablePath(); if (tablePath == null) { - tablePath = store.getTablePath(table.getTableName(), table.getDatabaseName()); + tablePath = getTablePath(table.getTableName(), table.getDatabaseName()); } TableInfo tableInfo = CarbonTable.builder() @@ -146,19 +173,27 @@ private void createTable(TableInfo tableInfo, boolean ifNotExists) throws IOExce } } - public void dropTable(TableIdentifier table) throws IOException { - String tablePath = store.getTablePath(table.getTableName(), table.getDatabaseName()); + public void dropTable(TableIdentifier table) throws CarbonException { + String tablePath = getTablePath(table.getTableName(), table.getDatabaseName()); cache.remove(tablePath); - FileFactory.deleteFile(tablePath); + try { + FileFactory.deleteFile(tablePath); + } catch (IOException e) { + throw new CarbonException(e); + } } - public CarbonTable getTable(TableIdentifier table) throws IOException { - String tablePath = store.getTablePath(table.getTableName(), table.getDatabaseName()); + public CarbonTable getTable(TableIdentifier table) throws CarbonException { + String tablePath = getTablePath(table.getTableName(), table.getDatabaseName()); if (cache.containsKey(tablePath)) { return cache.get(tablePath); } else { - org.apache.carbondata.format.TableInfo formatTableInfo = - CarbonUtil.readSchemaFile(CarbonTablePath.getSchemaFilePath(tablePath)); + org.apache.carbondata.format.TableInfo formatTableInfo = null; + try { + formatTableInfo = CarbonUtil.readSchemaFile(CarbonTablePath.getSchemaFilePath(tablePath)); + } catch (IOException e) { + throw new CarbonException(e); + } SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl(); TableInfo tableInfo = schemaConverter.fromExternalToWrapperTableInfo( formatTableInfo, table.getDatabaseName(), table.getTableName(), tablePath); @@ -168,4 +203,102 @@ public CarbonTable getTable(TableIdentifier table) throws IOException { return carbonTable; } } -} \ No newline at end of file + + public List listTable() throws CarbonException { + throw new UnsupportedOperationException(); + } + + public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { + throw new UnsupportedOperationException(); + } + + public void alterTable(TableIdentifier table, TableDescriptor newTable) throws CarbonException { + throw new UnsupportedOperationException(); + } + + public String getTablePath(String tableName, String databaseName) { + Objects.requireNonNull(tableName); + Objects.requireNonNull(databaseName); + return String.format("%s/%s", storeConf.storeLocation(), tableName); + } + + /** + * Prune data by using CarbonInputFormat.getSplit + * Return a mapping of host address to list of block. + * This should be invoked in driver side. + */ + static List pruneBlock(CarbonTable table, Expression filter) throws IOException { + Objects.requireNonNull(table); + JobConf jobConf = new JobConf(new Configuration()); + Job job = new Job(jobConf); + CarbonTableInputFormat format; + try { + // We just want to do pruning, so passing empty projection columns + format = CarbonInputFormatUtil.createCarbonTableInputFormat( + job, table, new String[0], filter, null, null, true); + } catch (InvalidConfigurationException e) { + throw new IOException(e.getMessage()); + } + + // We will do FG pruning in reader side, so don't do it here + CarbonInputFormat.setFgDataMapPruning(job.getConfiguration(), false); + List splits = format.getSplits(job); + List blockInfos = new ArrayList<>(splits.size()); + for (InputSplit split : splits) { + blockInfos.add((Distributable) split); + } + return blockInfos; + } + + /** + * Scan data and return matched rows. This should be invoked in worker side. + * @param table carbon table + * @param scan scan parameter + * @return matched rows + * @throws IOException if IO error occurs + */ + public static List scan(CarbonTable table, ScanRequest scan) throws IOException { + CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo(); + carbonTaskInfo.setTaskId(System.nanoTime()); + ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo); + + CarbonMultiBlockSplit mbSplit = scan.getSplit(); + long limit = scan.getLimit(); + QueryModel queryModel = createQueryModel(table, scan); + + LOGGER.info(String.format("[QueryId:%d] %s, number of block: %d", scan.getRequestId(), + queryModel.toString(), mbSplit.getAllSplits().size())); + + // read all rows by the reader + List rows = new LinkedList<>(); + try (CarbonRecordReader reader = new IndexedRecordReader(scan.getRequestId(), + table, queryModel)) { + reader.initialize(mbSplit, null); + + // loop to read required number of rows. + // By default, if user does not specify the limit value, limit is Long.MaxValue + long rowCount = 0; + while (reader.nextKeyValue() && rowCount < limit) { + rows.add(reader.getCurrentValue()); + rowCount++; + } + } catch (InterruptedException e) { + throw new IOException(e); + } + LOGGER.info(String.format("[QueryId:%d] scan completed, return %d rows", + scan.getRequestId(), rows.size())); + return rows; + } + + private static QueryModel createQueryModel(CarbonTable table, ScanRequest scan) { + String[] projectColumns = scan.getProjectColumns(); + Expression filter = null; + if (scan.getFilterExpression() != null) { + filter = scan.getFilterExpression(); + } + return new QueryModelBuilder(table) + .projectColumns(projectColumns) + .filterExpression(filter) + .build(); + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java index e354edc5006..3e44f78d199 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java @@ -19,18 +19,21 @@ import java.io.IOException; import java.net.BindException; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.sdk.store.Schedulable; import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.service.PruneService; +import org.apache.carbondata.sdk.store.service.StoreService; import org.apache.carbondata.sdk.store.util.StoreUtil; -import org.apache.carbondata.store.impl.rpc.PruneService; import org.apache.carbondata.store.impl.rpc.RegistryService; -import org.apache.carbondata.store.impl.rpc.ServiceFactory; -import org.apache.carbondata.store.impl.rpc.StoreService; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; @@ -41,7 +44,8 @@ * Master of CarbonStore. * It provides a Registry service and Prune service. */ -class Master { +@InterfaceAudience.Internal +public class Master { private static Master instance = null; @@ -51,26 +55,26 @@ class Master { private Configuration hadoopConf; private RPC.Server registryServer = null; private RPC.Server pruneServer = null; + private RPC.Server storeServer = null; // mapping of worker IP address to worker instance Map workers = new ConcurrentHashMap<>(); - private Master(StoreConf conf) { + public Master(StoreConf conf) { this.conf = conf; this.hadoopConf = conf.newHadoopConf(); } /** - * start registry service and listen on port passed in constructor + * start registry service */ - public void startRegistryService() throws IOException { + private void startRegistryService() throws IOException { if (registryServer == null) { - BindException exception; // we will try to create service at worse case 100 times int numTry = 100; String host = conf.masterHost(); - int port = conf.masterPort(); + int port = conf.registryServicePort(); LOGGER.info("building registry-service on " + host + ":" + port); RegistryService registryService = new RegistryServiceImpl(this); @@ -104,7 +108,7 @@ public void startRegistryService() throws IOException { } } - public void stopRegistryService() throws InterruptedException { + private void stopRegistryService() throws InterruptedException { if (registryServer != null) { registryServer.stop(); registryServer.join(); @@ -113,16 +117,15 @@ public void stopRegistryService() throws InterruptedException { } /** - * start registry service and listen on port passed in constructor + * start prune service */ - public void startPruneService() throws IOException { + private void startPruneService() throws IOException { if (pruneServer == null) { - BindException exception; // we will try to create service at worse case 100 times int numTry = 100; String host = conf.masterHost(); - int port = conf.prunePort(); + int port = conf.pruneServicePort(); LOGGER.info("building prune-service on " + host + ":" + port); PruneService pruneService = new PruneServiceImpl(); @@ -134,7 +137,7 @@ public void startPruneService() throws IOException { .setProtocol(PruneService.class) .setInstance(pruneService) .build(); - + ((PruneServiceImpl) pruneService).setScheduler(new Scheduler(this)); pruneServer.start(); numTry = 0; exception = null; @@ -156,7 +159,7 @@ public void startPruneService() throws IOException { } } - public void stopPruneService() throws InterruptedException { + private void stopPruneService() throws InterruptedException { if (pruneServer != null) { pruneServer.stop(); pruneServer.join(); @@ -164,10 +167,61 @@ public void stopPruneService() throws InterruptedException { } } + /** + * start store service + */ + private void startStoreService() throws IOException { + if (storeServer == null) { + BindException exception; + // we will try to create service at worse case 100 times + int numTry = 100; + String host = conf.masterHost(); + int port = conf.storeServicePort(); + LOGGER.info("building store-service on " + host + ":" + port); + + StoreService storeService = new StoreServiceImpl(conf); + do { + try { + storeServer = new RPC.Builder(hadoopConf) + .setBindAddress(host) + .setPort(port) + .setProtocol(StoreService.class) + .setInstance(storeService) + .build(); + + storeServer.start(); + numTry = 0; + exception = null; + } catch (BindException e) { + // port is occupied, increase the port number and try again + exception = e; + LOGGER.error(e, "start store-service failed"); + port = port + 1; + numTry = numTry - 1; + } + } while (numTry > 0); + if (exception != null) { + // we have tried many times, but still failed to find an available port + throw exception; + } + LOGGER.info("store-service started"); + } else { + LOGGER.info("store-service has already started"); + } + } + + private void stopStoreService() throws InterruptedException { + if (storeServer != null) { + storeServer.stop(); + storeServer.join(); + storeServer = null; + } + } + /** * A new searcher is trying to register, add it to the map and connect to this searcher */ - public RegisterWorkerResponse addWorker(RegisterWorkerRequest request) throws IOException { + RegisterWorkerResponse addWorker(RegisterWorkerRequest request) throws IOException { LOGGER.info( "Receive Register request from worker " + request.getHostAddress() + ":" + request.getPort() + " with " + request.getCores() + " cores"); @@ -178,9 +232,7 @@ public RegisterWorkerResponse addWorker(RegisterWorkerRequest request) throws IO "connecting to worker " + request.getHostAddress() + ":" + request.getPort() + ", workerId " + workerId); - StoreService searchService = ServiceFactory.createStoreService(workerAddress, workerPort); - addWorker( - new Schedulable(workerId, workerAddress, workerPort, request.getCores(), searchService)); + addWorker(new Schedulable(workerId, workerAddress, workerPort, request.getCores())); LOGGER.info("worker " + request + " registered"); return new RegisterWorkerResponse(workerId); } @@ -199,6 +251,15 @@ public static synchronized Master getInstance(StoreConf conf) { return instance; } + public void stopService() throws InterruptedException { + stopRegistryService(); + stopPruneService(); + } + + public List getAllWorkerAddresses() { + return new ArrayList<>(workers.keySet()); + } + public static void main(String[] args) throws InterruptedException, IOException { if (args.length != 2) { System.err.println("Usage: Master "); @@ -208,6 +269,7 @@ public static void main(String[] args) throws InterruptedException, IOException StoreUtil.initLog4j(args[0]); StoreConf conf = new StoreConf(args[1]); Master master = getInstance(conf); + master.startStoreService(); master.startRegistryService(); master.startPruneService(); Thread.sleep(Long.MAX_VALUE); diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java index 603a6012768..b56b0ae3491 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java @@ -24,11 +24,12 @@ import org.apache.carbondata.hadoop.CarbonInputSplit; import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; +import org.apache.carbondata.sdk.store.BlockScanUnit; import org.apache.carbondata.sdk.store.ScanUnit; -import org.apache.carbondata.store.impl.BlockScanUnit; -import org.apache.carbondata.store.impl.rpc.PruneService; -import org.apache.carbondata.store.impl.rpc.model.PruneRequest; -import org.apache.carbondata.store.impl.rpc.model.PruneResponse; +import org.apache.carbondata.sdk.store.Schedulable; +import org.apache.carbondata.sdk.store.service.PruneService; +import org.apache.carbondata.sdk.store.service.model.PruneRequest; +import org.apache.carbondata.sdk.store.service.model.PruneResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtocolSignature; @@ -37,17 +38,29 @@ public class PruneServiceImpl implements PruneService { + private Scheduler scheduler; + + public void setScheduler(Scheduler scheduler) { + this.scheduler = scheduler; + } + @Override public PruneResponse prune(PruneRequest request) throws IOException { Configuration hadoopConf = request.getHadoopConf(); Job job = new Job(hadoopConf); CarbonTableInputFormat format = new CarbonTableInputFormat(hadoopConf); List prunedResult = format.getSplits(job); - - List output = prunedResult.stream().map( - (Function) inputSplit -> - new BlockScanUnit((CarbonInputSplit) inputSplit) - ).collect(Collectors.toList()); + List output = + prunedResult.stream().map((Function) inputSplit -> { + String[] locations = ((CarbonInputSplit) inputSplit).preferredLocations(); + Schedulable worker; + if (locations.length == 0) { + worker = scheduler.pickNexWorker(); + } else { + worker = scheduler.pickWorker(locations[0]); + } + return new BlockScanUnit((CarbonInputSplit) inputSplit, worker); + }).collect(Collectors.toList()); return new PruneResponse(output); } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java index a494b313853..5f9cec7e567 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java @@ -17,79 +17,33 @@ package org.apache.carbondata.store.impl.master; -import java.io.IOException; import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.Schedulable; import org.apache.carbondata.sdk.store.exception.SchedulerException; -import org.apache.carbondata.store.impl.rpc.PruneService; -import org.apache.carbondata.store.impl.rpc.model.BaseResponse; -import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; -import org.apache.carbondata.store.impl.rpc.model.PruneRequest; -import org.apache.carbondata.store.impl.rpc.model.PruneResponse; -import org.apache.carbondata.store.impl.rpc.model.QueryResponse; -import org.apache.carbondata.store.impl.rpc.model.Scan; -import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; /** * [[Master]] uses Scheduler to pick a Worker to send request */ @InterfaceAudience.Internal -public class Scheduler { +class Scheduler { private static LogService LOGGER = LogServiceFactory.getLogService(Scheduler.class.getName()); private AtomicInteger count = new AtomicInteger(0); - private ExecutorService executors = Executors.newCachedThreadPool(); - private Master master; - - public Scheduler(StoreConf storeConf) throws IOException { - master = Master.getInstance(storeConf); - master.startRegistryService(); - } - - /** - * Pick a Worker according to the address and workload of the Worker - * Invoke the RPC and return Future result - */ - public Future sendRequestAsync( - final Schedulable worker, final Scan scan) { - LOGGER.info("sending search request to worker " + worker); - worker.workload.incrementAndGet(); - return executors.submit(new Callable() { - @Override public QueryResponse call() { - return worker.service.query(scan); - } - }); - } - - public BaseResponse sendRequest(final Schedulable worker, - final LoadDataRequest request) { - - LOGGER.info("sending load data request to worker " + worker); - worker.workload.incrementAndGet(); - return worker.service.loadData(request); - } - private PruneService pruneService; + private Master master; - public PruneResponse sendRequest(final PruneRequest request) throws IOException { - LOGGER.info("sending prune request to " + pruneService); - return pruneService.prune(request); + Scheduler(Master master) { + this.master = master; } - public Schedulable pickWorker(String splitAddress) { + Schedulable pickWorker(String splitAddress) { Schedulable worker = master.workers.get(splitAddress); // no local worker available, choose one worker randomly if (worker == null) { @@ -117,31 +71,12 @@ public Schedulable pickWorker(String splitAddress) { return worker; } - public Schedulable pickNexWorker() { + Schedulable pickNexWorker() { if (master.workers.size() == 0) { throw new SchedulerException("No worker is available"); } int index = count.getAndIncrement() % master.workers.size(); return new ArrayList<>(master.workers.values()).get(index); } - - public void stopAllWorkers() throws IOException { - for (Map.Entry entry : master.workers.entrySet()) { - try { - entry.getValue().service.shutdown(new ShutdownRequest("user")); - } catch (Throwable throwable) { - throw new IOException(throwable); - } - master.workers.remove(entry.getKey()); - } - } - - public void stopService() throws InterruptedException { - master.stopRegistryService(); - } - - public List getAllWorkerAddresses() { - return new ArrayList<>(master.workers.keySet()); - } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java new file mode 100644 index 00000000000..aceba74e9d1 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java @@ -0,0 +1,100 @@ +/* + * 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.carbondata.store.impl.master; + +import java.io.IOException; +import java.util.List; + +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.service.StoreService; +import org.apache.carbondata.store.impl.LocalCarbonStore; +import org.apache.carbondata.store.impl.TableManager; + +import org.apache.hadoop.ipc.ProtocolSignature; + +public class StoreServiceImpl implements StoreService { + + // TODO: simple implementation, load and scan inside master + private LocalCarbonStore localStore; + + private TableManager tableManager; + + StoreServiceImpl(StoreConf storeConf) { + localStore = new LocalCarbonStore(storeConf); + tableManager = new TableManager(storeConf); + } + + @Override + public void createTable(TableDescriptor descriptor) throws CarbonException { + tableManager.createTable(descriptor); + } + + @Override + public void dropTable(TableIdentifier identifier) throws CarbonException { + tableManager.dropTable(identifier); + } + + @Override + public CarbonTable getTable(TableIdentifier identifier) throws CarbonException { + return tableManager.getTable(identifier); + } + + @Override + public List listTable() throws CarbonException { + return tableManager.listTable(); + } + + @Override + public TableDescriptor getDescriptor(TableIdentifier identifier) throws CarbonException { + return tableManager.getDescriptor(identifier); + } + + @Override + public void alterTable(TableIdentifier identifier, TableDescriptor newTable) + throws CarbonException { + tableManager.alterTable(identifier, newTable); + } + + @Override + public void loadData(LoadDescriptor loadDescriptor) throws CarbonException { + localStore.loadData(loadDescriptor); + } + + @Override + public List scan(ScanDescriptor scanDescriptor) throws CarbonException { + return localStore.scan(scanDescriptor); + } + + @Override + public long getProtocolVersion(String protocol, long clientVersion) throws IOException { + return versionID; + } + + @Override + public ProtocolSignature getProtocolSignature(String protocol, long clientVersion, + int clientMethodsHash) throws IOException { + return null; + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java index ad8f22cd0c3..a9a14d5f1cc 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java @@ -29,21 +29,10 @@ @InterfaceAudience.Internal public class ServiceFactory { - public static StoreService createStoreService(String host, int port) throws IOException { - InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); - return RPC.getProxy( - StoreService.class, StoreService.versionID, address, new Configuration()); - } - public static RegistryService createRegistryService(String host, int port) throws IOException { InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); return RPC.getProxy( RegistryService.class, RegistryService.versionID, address, new Configuration()); } - public static PruneService createPruneService(String host, int port) throws IOException { - InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); - return RPC.getProxy( - PruneService.class, PruneService.versionID, address, new Configuration()); - } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/StoreService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/StoreService.java deleted file mode 100644 index 36702d991f0..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/StoreService.java +++ /dev/null @@ -1,40 +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.carbondata.store.impl.rpc; - -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.store.impl.rpc.model.BaseResponse; -import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; -import org.apache.carbondata.store.impl.rpc.model.QueryResponse; -import org.apache.carbondata.store.impl.rpc.model.Scan; -import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; -import org.apache.carbondata.store.impl.rpc.model.ShutdownResponse; - -import org.apache.hadoop.ipc.VersionedProtocol; - -@InterfaceAudience.Internal -public interface StoreService extends VersionedProtocol { - - long versionID = 1L; - - BaseResponse loadData(LoadDataRequest request); - - QueryResponse query(Scan scan); - - ShutdownResponse shutdown(ShutdownRequest request); -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java similarity index 86% rename from store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java index 981ab7672a6..23b9369bb09 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/RequestHandler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java @@ -37,13 +37,13 @@ import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.service.model.BaseResponse; +import org.apache.carbondata.sdk.store.service.model.LoadDataRequest; +import org.apache.carbondata.sdk.store.service.model.ScanRequest; +import org.apache.carbondata.sdk.store.service.model.ScanResponse; import org.apache.carbondata.sdk.store.util.StoreUtil; -import org.apache.carbondata.store.impl.CarbonStoreBase; import org.apache.carbondata.store.impl.Status; -import org.apache.carbondata.store.impl.rpc.model.BaseResponse; -import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; -import org.apache.carbondata.store.impl.rpc.model.QueryResponse; -import org.apache.carbondata.store.impl.rpc.model.Scan; +import org.apache.carbondata.store.impl.TableManager; import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; import org.apache.carbondata.store.impl.rpc.model.ShutdownResponse; @@ -58,27 +58,27 @@ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; /** - * It handles request from master. + * It handles data request from client. */ @InterfaceAudience.Internal -class RequestHandler { +class DataRequestHandler { private StoreConf storeConf; private Configuration hadoopConf; - RequestHandler(StoreConf conf, Configuration hadoopConf) { + DataRequestHandler(StoreConf conf, Configuration hadoopConf) { this.storeConf = conf; this.hadoopConf = hadoopConf; } private static final LogService LOGGER = - LogServiceFactory.getLogService(RequestHandler.class.getName()); + LogServiceFactory.getLogService(DataRequestHandler.class.getName()); - QueryResponse handleScan(Scan scan) { + ScanResponse handleScan(ScanRequest scan) { try { LOGGER.info(String.format("[QueryId:%d] receive search request", scan.getRequestId())); CarbonTable table = CarbonTable.buildFromTableInfo(scan.getTableInfo()); - List rows = CarbonStoreBase.scan(table, scan); + List rows = TableManager.scan(table, scan); LOGGER.info(String.format("[QueryId:%d] sending success response", scan.getRequestId())); return createSuccessResponse(scan, rows); } catch (IOException e) { @@ -99,22 +99,22 @@ ShutdownResponse handleShutdown(ShutdownRequest request) { /** * create a failure response */ - private QueryResponse createFailureResponse(Scan scan, Throwable throwable) { - return new QueryResponse(scan.getRequestId(), Status.FAILURE.ordinal(), + private ScanResponse createFailureResponse(ScanRequest scan, Throwable throwable) { + return new ScanResponse(scan.getRequestId(), Status.FAILURE.ordinal(), throwable.getMessage(), new Object[0][]); } /** * create a success response with result rows */ - private QueryResponse createSuccessResponse(Scan scan, List rows) { + private ScanResponse createSuccessResponse(ScanRequest scan, List rows) { Iterator itor = rows.iterator(); Object[][] output = new Object[rows.size()][]; int i = 0; while (itor.hasNext()) { output[i++] = itor.next().getData(); } - return new QueryResponse(scan.getRequestId(), Status.SUCCESS.ordinal(), "", output); + return new ScanResponse(scan.getRequestId(), Status.SUCCESS.ordinal(), "", output); } public BaseResponse handleLoadData(LoadDataRequest request) { diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/StoreServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java similarity index 60% rename from store/core/src/main/java/org/apache/carbondata/store/impl/worker/StoreServiceImpl.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java index 26f252c838b..fd50024eb4f 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/StoreServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java @@ -20,25 +20,21 @@ import java.io.IOException; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.store.impl.rpc.StoreService; -import org.apache.carbondata.store.impl.rpc.model.BaseResponse; -import org.apache.carbondata.store.impl.rpc.model.LoadDataRequest; -import org.apache.carbondata.store.impl.rpc.model.QueryResponse; -import org.apache.carbondata.store.impl.rpc.model.Scan; -import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; -import org.apache.carbondata.store.impl.rpc.model.ShutdownResponse; +import org.apache.carbondata.sdk.store.service.DataService; +import org.apache.carbondata.sdk.store.service.model.BaseResponse; +import org.apache.carbondata.sdk.store.service.model.LoadDataRequest; +import org.apache.carbondata.sdk.store.service.model.ScanRequest; +import org.apache.carbondata.sdk.store.service.model.ScanResponse; import org.apache.hadoop.ipc.ProtocolSignature; @InterfaceAudience.Internal -public class StoreServiceImpl implements StoreService { +public class DataServiceImpl implements DataService { - private Worker worker; - RequestHandler handler; + private DataRequestHandler handler; - public StoreServiceImpl(Worker worker) { - this.worker = worker; - this.handler = new RequestHandler(worker.getConf(), worker.getHadoopConf()); + DataServiceImpl(Worker worker) { + this.handler = new DataRequestHandler(worker.getConf(), worker.getHadoopConf()); } @Override @@ -47,15 +43,10 @@ public BaseResponse loadData(LoadDataRequest request) { } @Override - public QueryResponse query(Scan scan) { + public ScanResponse scan(ScanRequest scan) { return handler.handleScan(scan); } - @Override - public ShutdownResponse shutdown(ShutdownRequest request) { - return handler.handleShutdown(request); - } - @Override public long getProtocolVersion(String protocol, long clientVersion) throws IOException { return versionID; @@ -67,11 +58,4 @@ public ProtocolSignature getProtocolSignature(String protocol, long clientVersio return null; } - public Worker getWorker() { - return worker; - } - - public void setWorker(Worker worker) { - this.worker = worker; - } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java index 101cb276c54..5349456ed07 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java @@ -24,10 +24,10 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.service.DataService; import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.rpc.RegistryService; import org.apache.carbondata.store.impl.rpc.ServiceFactory; -import org.apache.carbondata.store.impl.rpc.StoreService; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; @@ -66,15 +66,15 @@ private void startService() throws IOException { int coreNum = conf.workerCoreNum(); String host = conf.workerHost(); int port = conf.workerPort(); - StoreService queryService = new StoreServiceImpl(this); + DataService dataService = new DataServiceImpl(this); do { try { server = new RPC.Builder(hadoopConf) .setNumHandlers(coreNum) .setBindAddress(host) .setPort(port) - .setProtocol(StoreService.class) - .setInstance(queryService) + .setProtocol(DataService.class) + .setInstance(dataService) .build(); server.start(); @@ -116,9 +116,11 @@ private void stopService() throws InterruptedException { } private void registerToMaster() throws IOException { - LOGGER.info("trying to register to master " + conf.masterHost() + ":" + conf.masterPort()); + LOGGER.info("trying to register to master " + + conf.masterHost() + ":" + conf.registryServicePort()); if (registry == null) { - registry = ServiceFactory.createRegistryService(conf.masterHost(), conf.masterPort()); + registry = ServiceFactory.createRegistryService( + conf.masterHost(), conf.registryServicePort()); } RegisterWorkerRequest request = new RegisterWorkerRequest(conf.workerHost(), conf.workerPort(), conf.workerCoreNum()); diff --git a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/RemoteCarbonStoreTest.java similarity index 87% rename from store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java rename to store/core/src/test/java/org/apache/carbondata/store/RemoteCarbonStoreTest.java index d507128803e..f9c49cb4354 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/RemoteCarbonStoreTest.java @@ -27,14 +27,15 @@ import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.LiteralExpression; import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; +import org.apache.carbondata.sdk.store.CarbonStore; +import org.apache.carbondata.sdk.store.CarbonStoreFactory; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.sdk.store.CarbonStore; -import org.apache.carbondata.sdk.store.CarbonStoreFactory; import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.store.impl.master.Master; import org.apache.carbondata.store.impl.worker.Worker; import org.junit.After; @@ -44,30 +45,42 @@ import org.junit.BeforeClass; import org.junit.Test; -public class DistributedCarbonStoreTest { +public class RemoteCarbonStoreTest { private static String projectFolder; private static CarbonStore store; @BeforeClass - public static void beforeAll() throws IOException, CarbonException { - projectFolder = new File(DistributedCarbonStoreTest.class.getResource("/").getPath() + + public static void beforeAll() throws IOException, CarbonException, InterruptedException { + projectFolder = new File(RemoteCarbonStoreTest.class.getResource("/").getPath() + "../../../../").getCanonicalPath(); + String confFile = projectFolder + "/store/conf/store.conf"; StoreConf storeConf = new StoreConf(confFile); - store = CarbonStoreFactory.getDistributedStore("DistributedCarbonStoreTest", storeConf); - projectFolder = new File(LocalCarbonStoreTest.class.getResource("/").getPath() + "../../../../") - .getCanonicalPath(); + new Thread(() -> { + try { + Master.main(new String[]{"", confFile}); + } catch (InterruptedException | IOException e) { + throw new RuntimeException("failed to start master"); + } + }).start(); + Thread.sleep(1000); // start worker Worker worker = new Worker(storeConf); worker.start(); + + Thread.sleep(1000); + + store = CarbonStoreFactory.getRemoteStore("RemoteCarbonStoreTest", storeConf); } @AfterClass public static void afterAll() throws IOException { - store.close(); + if (store != null) { + store.close(); + } } @Before @@ -81,7 +94,7 @@ public void verifyDMFile() { } @Test - public void testSelect() throws IOException, CarbonException { + public void testSelect() throws CarbonException { TableIdentifier tableIdentifier = new TableIdentifier("table_1", "default"); store.dropTable(tableIdentifier); TableDescriptor descriptor = TableDescriptor diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java index fdd1f5abd4b..771896be7f3 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/AvroCarbonWriter.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.Random; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.logging.LogService; @@ -30,6 +31,7 @@ import org.apache.carbondata.core.metadata.datatype.DataType; import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.metadata.datatype.StructField; +import org.apache.carbondata.core.metadata.datatype.StructType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.hadoop.api.CarbonTableOutputFormat; import org.apache.carbondata.hadoop.internal.ObjectArrayWritable; @@ -239,9 +241,9 @@ private static Field prepareFields(Schema.Field avroField) { return new Field(FieldName, DataTypes.DOUBLE); case RECORD: // recursively get the sub fields - ArrayList structSubFields = new ArrayList<>(); + ArrayList structSubFields = new ArrayList<>(); for (Schema.Field avroSubField : childSchema.getFields()) { - StructField structField = prepareSubFields(avroSubField.name(), avroSubField.schema()); + Field structField = prepareSubFields(avroSubField.name(), avroSubField.schema()); if (structField != null) { structSubFields.add(structField); } @@ -249,9 +251,9 @@ private static Field prepareFields(Schema.Field avroField) { return new Field(FieldName, "struct", structSubFields); case ARRAY: // recursively get the sub fields - ArrayList arraySubField = new ArrayList<>(); + ArrayList arraySubField = new ArrayList<>(); // array will have only one sub field. - StructField structField = prepareSubFields("val", childSchema.getElementType()); + Field structField = prepareSubFields("val", childSchema.getElementType()); if (structField != null) { arraySubField.add(structField); return new Field(FieldName, "array", arraySubField); @@ -266,51 +268,51 @@ private static Field prepareFields(Schema.Field avroField) { } } - private static StructField prepareSubFields(String FieldName, Schema childSchema) { + private static Field prepareSubFields(String FieldName, Schema childSchema) { Schema.Type type = childSchema.getType(); LogicalType logicalType = childSchema.getLogicalType(); switch (type) { case BOOLEAN: - return new StructField(FieldName, DataTypes.BOOLEAN); + return new Field(FieldName, DataTypes.BOOLEAN); case INT: if (logicalType instanceof LogicalTypes.Date) { - return new StructField(FieldName, DataTypes.DATE); + return new Field(FieldName, DataTypes.DATE); } else { LOGGER.warn("Unsupported logical type. Considering Data Type as INT for " + childSchema .getName()); - return new StructField(FieldName, DataTypes.INT); + return new Field(FieldName, DataTypes.INT); } case LONG: if (logicalType instanceof LogicalTypes.TimestampMillis || logicalType instanceof LogicalTypes.TimestampMicros) { - return new StructField(FieldName, DataTypes.TIMESTAMP); + return new Field(FieldName, DataTypes.TIMESTAMP); } else { LOGGER.warn("Unsupported logical type. Considering Data Type as LONG for " + childSchema .getName()); - return new StructField(FieldName, DataTypes.LONG); + return new Field(FieldName, DataTypes.LONG); } case DOUBLE: - return new StructField(FieldName, DataTypes.DOUBLE); + return new Field(FieldName, DataTypes.DOUBLE); case STRING: - return new StructField(FieldName, DataTypes.STRING); + return new Field(FieldName, DataTypes.STRING); case FLOAT: - return new StructField(FieldName, DataTypes.DOUBLE); + return new Field(FieldName, DataTypes.DOUBLE); case RECORD: // recursively get the sub fields - ArrayList structSubFields = new ArrayList<>(); + ArrayList structSubFields = new ArrayList<>(); for (Schema.Field avroSubField : childSchema.getFields()) { - StructField structField = prepareSubFields(avroSubField.name(), avroSubField.schema()); + Field structField = prepareSubFields(avroSubField.name(), avroSubField.schema()); if (structField != null) { structSubFields.add(structField); } } - return (new StructField(FieldName, DataTypes.createStructType(structSubFields))); + return (new Field(FieldName, createStructType(structSubFields))); case ARRAY: // recursively get the sub fields // array will have only one sub field. DataType subType = getMappingDataTypeForArrayRecord(childSchema.getElementType()); if (subType != null) { - return (new StructField(FieldName, DataTypes.createArrayType(subType))); + return (new Field(FieldName, DataTypes.createArrayType(subType))); } else { return null; } @@ -322,6 +324,14 @@ private static StructField prepareSubFields(String FieldName, Schema childSchema } } + private static StructType createStructType(List fields) { + List f = fields.stream().map(field -> + new StructField(field.getFieldName(), field.getDataType(), + createStructType(field.getChildren()).getFields()) + ).collect(Collectors.toList()); + return DataTypes.createStructType(f); + } + private static DataType getMappingDataTypeForArrayRecord(Schema childSchema) { LogicalType logicalType = childSchema.getLogicalType(); switch (childSchema.getType()) { @@ -360,14 +370,14 @@ private static DataType getMappingDataTypeForArrayRecord(Schema childSchema) { return DataTypes.DOUBLE; case RECORD: // recursively get the sub fields - ArrayList structSubFields = new ArrayList<>(); + ArrayList structSubFields = new ArrayList<>(); for (Schema.Field avroSubField : childSchema.getFields()) { - StructField structField = prepareSubFields(avroSubField.name(), avroSubField.schema()); + Field structField = prepareSubFields(avroSubField.name(), avroSubField.schema()); if (structField != null) { structSubFields.add(structField); } } - return DataTypes.createStructType(structSubFields); + return createStructType(structSubFields); case ARRAY: // array will have only one sub field. DataType subType = getMappingDataTypeForArrayRecord(childSchema.getElementType()); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java index a9d725ff2bf..92ed0d8fa03 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java @@ -513,8 +513,8 @@ public static void buildTableSchema(Field[] fields, TableSchemaBuilder tableSche } else if (field.getDataType().getName().equalsIgnoreCase("STRUCT")) { // Loop through the inner columns and for a StructData List structFieldsArray = - new ArrayList(field.getChildren().size()); - for (StructField childFld : field.getChildren()) { + new ArrayList<>(field.getChildren().size()); + for (Field childFld : field.getChildren()) { structFieldsArray .add(new StructField(childFld.getFieldName(), childFld.getDataType())); } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java index add10c1b16d..0d70c3bd23a 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Field.java @@ -17,13 +17,13 @@ package org.apache.carbondata.sdk.file; +import java.io.Serializable; import java.util.LinkedList; import java.util.List; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.metadata.datatype.DataType; -import org.apache.carbondata.core.metadata.datatype.StructField; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; import org.apache.carbondata.core.util.DataTypeUtil; @@ -32,11 +32,11 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public class Field { +public class Field implements Serializable { private String name; private DataType type; - private List children; + private List children; private String parent; private String storeType = "columnnar"; private int schemaOrdinal = -1; @@ -54,11 +54,11 @@ public Field(String name, String type) { this(name, DataTypeUtil.valueOf(type)); } - public Field(String name, String type, List fields) { + public Field(String name, String type, List fields) { this(name, DataTypeUtil.valueOf(type), fields); } - public Field(String name, DataType type, List fields) { + public Field(String name, DataType type, List fields) { this.name = name; this.type = type; this.children = fields; @@ -91,11 +91,11 @@ public DataType getDataType() { return type; } - public List getChildren() { + public List getChildren() { return children; } - public void setChildren(List children) { + public void setChildren(List children) { this.children = children; } @@ -150,4 +150,5 @@ public String getColumnComment() { public void setColumnComment(String columnComment) { this.columnComment = columnComment; } + } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java index c9622e10d36..075ae716262 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/Schema.java @@ -18,6 +18,7 @@ package org.apache.carbondata.sdk.file; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; @@ -44,7 +45,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public class Schema { +public class Schema implements Serializable { private Field[] fields; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java similarity index 69% rename from store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java index 771cb147418..f795bff537a 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java @@ -15,29 +15,48 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl; +package org.apache.carbondata.sdk.store; import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.hadoop.CarbonInputSplit; -import org.apache.carbondata.sdk.store.ScanUnit; +/** + * It contains a block to scan, and a destination worker who should scan it + */ +@InterfaceAudience.Internal public class BlockScanUnit implements ScanUnit { + + // the data block to scan private CarbonInputSplit inputSplit; + // the worker who should scan this unit + private Schedulable schedulable; + public BlockScanUnit() { } - public BlockScanUnit(CarbonInputSplit inputSplit) { + public BlockScanUnit(CarbonInputSplit inputSplit, Schedulable schedulable) { this.inputSplit = inputSplit; + this.schedulable = schedulable; } public CarbonInputSplit getInputSplit() { return inputSplit; } + public Schedulable getSchedulable() { + return schedulable; + } + + @Override + public String[] preferredLocations() { + return inputSplit.preferredLocations(); + } + @Override public void write(DataOutput out) throws IOException { inputSplit.write(out); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java index 580070dd1fb..d666331a868 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -141,19 +141,11 @@ default Mutator newMutator() throws CarbonException { */ List scan(ScanDescriptor select) throws CarbonException; - /** - * Scan a Table and return matched rows - * @param select descriptor for select operation, including required column, filter, etc - * @return matched rows - * @throws CarbonException if any error occurs - */ - List scan(ScanDescriptor select, SelectOption option) throws CarbonException; - /** * @return a new Scanner * @throws CarbonException if any error occurs */ - Scanner newScanner() throws CarbonException; + Scanner newScanner(TableIdentifier tableIdentifier) throws CarbonException; /** * @return a new Fetcher diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java index 761a2113777..64019786d36 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java @@ -34,23 +34,23 @@ @InterfaceAudience.User @InterfaceStability.Unstable public class CarbonStoreFactory { - private static Map distributedStores = new ConcurrentHashMap<>(); + private static Map remoteStore = new ConcurrentHashMap<>(); private static Map localStores = new ConcurrentHashMap<>(); private CarbonStoreFactory() { } - public static CarbonStore getDistributedStore(String storeName, StoreConf storeConf) + public static CarbonStore getRemoteStore(String storeName, StoreConf storeConf) throws CarbonException { - if (distributedStores.containsKey(storeName)) { - return distributedStores.get(storeName); + if (remoteStore.containsKey(storeName)) { + return remoteStore.get(storeName); } // create a new instance try { - String className = "org.apache.carbondata.store.impl.DistributedCarbonStore"; + String className = "org.apache.carbondata.sdk.store.RemoteCarbonStore"; CarbonStore store = createCarbonStore(storeConf, className); - distributedStores.put(storeName, store); + remoteStore.put(storeName, store); return store; } catch (ClassNotFoundException | IllegalAccessException | InvocationTargetException | InstantiationException e) { @@ -59,9 +59,9 @@ public static CarbonStore getDistributedStore(String storeName, StoreConf storeC } public static void removeDistributedStore(String storeName) throws IOException { - if (distributedStores.containsKey(storeName)) { - distributedStores.get(storeName).close(); - distributedStores.remove(storeName); + if (remoteStore.containsKey(storeName)) { + remoteStore.get(storeName).close(); + remoteStore.remove(storeName); } } @@ -83,13 +83,6 @@ public static CarbonStore getLocalStore(String storeName, StoreConf storeConf) } } - public static void removeLocalStore(String storeName) throws IOException { - if (localStores.containsKey(storeName)) { - localStores.get(storeName).close(); - localStores.remove(storeName); - } - } - private static CarbonStore createCarbonStore(StoreConf storeConf, String className) throws ClassNotFoundException, InstantiationException, IllegalAccessException, InvocationTargetException { diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java new file mode 100644 index 00000000000..8f0b4edfa46 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java @@ -0,0 +1,44 @@ +/* + * 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.carbondata.sdk.store; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.carbondata.sdk.store.service.DataService; +import org.apache.carbondata.sdk.store.service.ServiceFactory; + +public class DataServicePool { + private static final Map dataServicePool = new ConcurrentHashMap<>(); + + private DataServicePool() { + } + + static DataService getOrCreateDataService(Schedulable schedulable) throws IOException { + DataService service = dataServicePool.getOrDefault( + schedulable, + ServiceFactory.createDataService(schedulable.getAddress(), schedulable.getPort())); + dataServicePool.putIfAbsent(schedulable, service); + return service; + } + + public static void releaseAll() { + dataServicePool.forEach((schedulable, service) -> ServiceFactory.stopDataService(service)); + } +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java new file mode 100644 index 00000000000..64bc7485025 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java @@ -0,0 +1,130 @@ +/* + * 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.carbondata.sdk.store; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.service.ServiceFactory; +import org.apache.carbondata.sdk.store.service.StoreService; + +/** + * A CarbonStore that leverage multiple servers via RPC calls (Master and Workers) + */ +public class RemoteCarbonStore implements CarbonStore { + + private static final LogService LOGGER = + LogServiceFactory.getLogService(RemoteCarbonStore.class.getCanonicalName()); + + private static final long versionID = 1L; + + private StoreService storeService; + private StoreConf storeConf; + private Map tableCache = new ConcurrentHashMap<>(); + + public RemoteCarbonStore(StoreConf conf) throws IOException { + this.storeService = + ServiceFactory.createStoreService(conf.masterHost(), conf.storeServicePort()); + this.storeConf = conf; + } + + @Override + public void createTable(TableDescriptor descriptor) throws CarbonException { + try { + storeService.createTable(descriptor); + } catch (Exception e) { + System.out.println(e.getMessage()); + } + } + + @Override + public void dropTable(TableIdentifier table) throws CarbonException { + try { + storeService.dropTable(table); + } catch (Exception e) { + System.out.println(e.getMessage()); + } + } + + @Override + public CarbonTable getTable(TableIdentifier table) throws CarbonException { + return storeService.getTable(table); + } + + @Override + public List listTable() throws CarbonException { + return storeService.listTable(); + } + + @Override + public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { + return storeService.getDescriptor(table); + } + + @Override + public void alterTable(TableIdentifier table, TableDescriptor newTable) throws CarbonException { + storeService.alterTable(table, newTable); + } + + @Override + public void loadData(LoadDescriptor load) throws CarbonException { + storeService.loadData(load); + } + + @Override + public Loader newLoader(LoadDescriptor load) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public List scan(ScanDescriptor select) throws CarbonException { + try { + return storeService.scan(select); + } catch (Exception e) { + System.out.println(e.getMessage()); + return null; + } + } + + @Override + public Scanner newScanner(TableIdentifier identifier) throws CarbonException { + CarbonTable table = tableCache.getOrDefault(identifier, storeService.getTable(identifier)); + tableCache.putIfAbsent(identifier, table); + try { + return new ScannerImpl(storeConf, table.getTableInfo()); + } catch (IOException e) { + throw new CarbonException(e); + } + } + + @Override + public void close() throws IOException { + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowMajorResultBatch.java similarity index 93% rename from store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowMajorResultBatch.java index 9755983824d..534750ae7b6 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowMajorResultBatch.java @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl; +package org.apache.carbondata.sdk.store; import java.util.Iterator; import java.util.List; import java.util.Objects; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.sdk.store.ResultBatch; public class RowMajorResultBatch implements ResultBatch { diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java index b4385df7bde..366a7d3a023 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java @@ -66,7 +66,7 @@ public interface Scanner extends Serializable { * @return scan result, the result is returned in batch * @throws CarbonException if any error occurs */ - Iterator> scan(ScanUnit input, ScanDescriptor select, + Iterator> scan(ScanUnit input, ScanDescriptor select, SelectOption option) throws CarbonException; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java new file mode 100644 index 00000000000..8c1fd1b25d0 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java @@ -0,0 +1,122 @@ +/* + * 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.carbondata.sdk.store; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; + +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; +import org.apache.carbondata.hadoop.api.CarbonInputFormat; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.service.DataService; +import org.apache.carbondata.sdk.store.service.PruneService; +import org.apache.carbondata.sdk.store.service.ServiceFactory; +import org.apache.carbondata.sdk.store.service.model.PruneRequest; +import org.apache.carbondata.sdk.store.service.model.PruneResponse; +import org.apache.carbondata.sdk.store.service.model.ScanRequest; +import org.apache.carbondata.sdk.store.service.model.ScanResponse; + +import org.apache.hadoop.conf.Configuration; + +class ScannerImpl implements Scanner { + private static final LogService LOGGER = + LogServiceFactory.getLogService(ScannerImpl.class.getCanonicalName()); + + private PruneService pruneService; + private TableInfo tableInfo; + + ScannerImpl(StoreConf conf, TableInfo tableInfo) throws IOException { + this.pruneService = ServiceFactory.createPruneService( + conf.masterHost(), conf.registryServicePort()); + this.tableInfo = tableInfo; + } + + /** + * Trigger a RPC to Carbon Master to do pruning + * @param table table identifier + * @param filterExpression expression of filter predicate given by user + * @return list of ScanUnit + * @throws CarbonException if any error occurs + */ + @Override + public List prune(TableIdentifier table, Expression filterExpression) + throws CarbonException { + try { + Configuration configuration = new Configuration(); + CarbonInputFormat.setTableName(configuration, table.getTableName()); + CarbonInputFormat.setDatabaseName(configuration, table.getDatabaseName()); + CarbonInputFormat.setFilterPredicates(configuration, filterExpression); + PruneRequest request = new PruneRequest(configuration); + PruneResponse response = pruneService.prune(request); + return response.getScanUnits(); + } catch (IOException e) { + throw new CarbonException(e); + } + } + + /** + * Execute scan by firing RPC call to worker, return the result rows + * @param input one scan unit + * @param descriptor parameter for scan + * @param option scan options + * @return result batch + * @throws CarbonException if any error occurs + */ + @Override + public Iterator> scan(ScanUnit input, ScanDescriptor descriptor, + SelectOption option) throws CarbonException { + List toBeScan = new ArrayList<>(); + if (input instanceof BlockScanUnit) { + toBeScan.add(((BlockScanUnit) input).getInputSplit()); + } else { + throw new CarbonException(input.getClass().getName() + " is not supported"); + } + int queryId = new Random().nextInt(); + CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(toBeScan, input.preferredLocations()); + try { + ScanRequest request = new ScanRequest(queryId, split, tableInfo, descriptor.getProjection(), + descriptor.getFilter(), descriptor.getLimit()); + DataService dataService = + DataServicePool.getOrCreateDataService(((BlockScanUnit) input).getSchedulable()); + ScanResponse response = dataService.scan(request); + List rows = Arrays.stream(response.getRows()) + .map(CarbonRow::new) + .collect(Collectors.toList()); + + return Collections.singletonList(new RowMajorResultBatch(rows)).iterator(); + } catch (IOException e) { + throw new CarbonException(e); + } + } + +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Schedulable.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java similarity index 89% rename from store/core/src/main/java/org/apache/carbondata/store/impl/master/Schedulable.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java index 57d41f07c96..95cf3d047a6 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Schedulable.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.master; +package org.apache.carbondata.sdk.store; import java.util.concurrent.atomic.AtomicInteger; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.store.impl.rpc.StoreService; @InterfaceAudience.Internal public class Schedulable { @@ -29,15 +28,13 @@ public class Schedulable { private String address; private int port; private int cores; - public StoreService service; public AtomicInteger workload; - public Schedulable(String id, String address, int port, int cores, StoreService service) { + public Schedulable(String id, String address, int port, int cores) { this.id = id; this.address = address; this.port = port; this.cores = cores; - this.service = service; this.workload = new AtomicInteger(); } @@ -65,7 +62,7 @@ public void setPort(int port) { this.port = port; } - int getCores() { + public int getCores() { return cores; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java index 89879a7b06b..c6c7cf6579e 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/conf/StoreConf.java @@ -44,8 +44,9 @@ public class StoreConf implements Serializable, Writable { public static final String WORKER_PORT = "carbon.worker.port"; public static final String WORKER_CORE_NUM = "carbon.worker.core.num"; public static final String MASTER_HOST = "carbon.master.host"; - public static final String MASTER_PORT = "carbon.master.port"; + public static final String REGISTRY_PORT = "carbon.master.registry.port"; public static final String PRUNE_PORT = "carbon.master.prune.port"; + public static final String STORE_PORT = "carbon.master.store.port"; public static final String STORE_TEMP_LOCATION = "carbon.store.temp.location"; public static final String STORE_LOCATION = "carbon.store.location"; @@ -109,14 +110,18 @@ public String masterHost() { return stringValue(MASTER_HOST); } - public int masterPort() { - return intValue(MASTER_PORT); + public int registryServicePort() { + return intValue(REGISTRY_PORT); } - public int prunePort() { + public int pruneServicePort() { return intValue(PRUNE_PORT); } + public int storeServicePort() { + return intValue(STORE_PORT); + } + public String workerHost() { return stringValue(WORKER_HOST); } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java index b05bfb6c616..fc4308e60c4 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/LoadDescriptor.java @@ -17,6 +17,9 @@ package org.apache.carbondata.sdk.store.descriptor; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -24,9 +27,11 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.hadoop.io.Writable; + @InterfaceAudience.User @InterfaceStability.Evolving -public class LoadDescriptor { +public class LoadDescriptor implements Writable { private TableIdentifier table; private String inputPath; @@ -78,6 +83,31 @@ public void setOverwrite(boolean overwrite) { isOverwrite = overwrite; } + @Override + public void write(DataOutput out) throws IOException { + table.write(out); + out.writeUTF(inputPath); + out.writeInt(options.size()); + for (Map.Entry entry : options.entrySet()) { + out.writeUTF(entry.getKey()); + out.writeUTF(entry.getValue()); + } + out.writeBoolean(isOverwrite); + } + + @Override + public void readFields(DataInput in) throws IOException { + table = new TableIdentifier(); + table.readFields(in); + inputPath = in.readUTF(); + int size = in.readInt(); + options = new HashMap<>(size); + for (int i = 0; i < size; i++) { + options.put(in.readUTF(), in.readUTF()); + } + isOverwrite = in.readBoolean(); + } + public static class Builder { private LoadDescriptor load; private Map options; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java index fa562e1f796..f863f1db3b2 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java @@ -17,15 +17,21 @@ package org.apache.carbondata.sdk.store.descriptor; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.Objects; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.core.util.ObjectSerializationUtil; + +import org.apache.hadoop.io.Writable; @InterfaceAudience.User @InterfaceStability.Evolving -public class ScanDescriptor { +public class ScanDescriptor implements Writable { private TableIdentifier table; private String[] projection; @@ -77,6 +83,35 @@ public void setLimit(long limit) { this.limit = limit; } + @Override + public void write(DataOutput out) throws IOException { + table.write(out); + out.writeInt(projection.length); + for (String s : projection) { + out.writeUTF(s); + } + out.writeBoolean(filter != null); + if (filter != null) { + out.writeUTF(ObjectSerializationUtil.convertObjectToString(filter)); + } + out.writeLong(limit); + } + + @Override + public void readFields(DataInput in) throws IOException { + table = new TableIdentifier(); + table.readFields(in); + int size = in.readInt(); + projection = new String[size]; + for (int i = 0; i < size; i++) { + projection[i] = in.readUTF(); + } + if (in.readBoolean()) { + filter = (Expression) ObjectSerializationUtil.convertStringToObject(in.readUTF()); + } + limit = in.readLong(); + } + public static class Builder { private ScanDescriptor select; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java index 21ef8813c90..bdd5948d3fc 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableDescriptor.java @@ -17,6 +17,9 @@ package org.apache.carbondata.sdk.store.descriptor; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -24,13 +27,16 @@ import java.util.Objects; import org.apache.carbondata.core.metadata.datatype.DataType; +import org.apache.carbondata.core.util.ObjectSerializationUtil; import org.apache.carbondata.sdk.file.Field; import org.apache.carbondata.sdk.file.Schema; -public class TableDescriptor { +import org.apache.hadoop.io.Writable; + +public class TableDescriptor implements Writable { - private boolean ifNotExists; private TableIdentifier table; + private boolean ifNotExists; private String tablePath; private Schema schema; private Map properties; @@ -99,6 +105,40 @@ public String getTablePath() { return tablePath; } + @Override + public void write(DataOutput out) throws IOException { + table.write(out); + out.writeBoolean(ifNotExists); + out.writeBoolean(tablePath != null); + if (tablePath != null) { + out.writeUTF(tablePath); + } + out.writeUTF(ObjectSerializationUtil.convertObjectToString(schema)); + out.writeInt(properties.size()); + for (Map.Entry entry : properties.entrySet()) { + out.writeUTF(entry.getKey()); + out.writeUTF(entry.getValue()); + } + out.writeUTF(comment); + } + + @Override + public void readFields(DataInput in) throws IOException { + table = new TableIdentifier(); + table.readFields(in); + ifNotExists = in.readBoolean(); + if (in.readBoolean()) { + tablePath = in.readUTF(); + } + schema = (Schema) ObjectSerializationUtil.convertStringToObject(in.readUTF()); + int size = in.readInt(); + properties = new HashMap<>(size); + for (int i = 0; i < size; i++) { + properties.put(in.readUTF(), in.readUTF()); + } + comment = in.readUTF(); + } + public static class Builder { private TableDescriptor table; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java index 59bf32bdb03..c67a748f815 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/TableIdentifier.java @@ -17,17 +17,25 @@ package org.apache.carbondata.sdk.store.descriptor; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.io.Serializable; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.hadoop.io.Writable; + @InterfaceAudience.User @InterfaceStability.Evolving -public class TableIdentifier implements Serializable { +public class TableIdentifier implements Serializable, Writable { private String tableName; private String databaseName; + public TableIdentifier() { + } + public TableIdentifier(String tableName, String databaseName) { this.tableName = tableName; this.databaseName = databaseName; @@ -41,4 +49,15 @@ public String getDatabaseName() { return databaseName; } + @Override + public void write(DataOutput out) throws IOException { + out.writeUTF(databaseName); + out.writeUTF(tableName); + } + + @Override + public void readFields(DataInput in) throws IOException { + this.databaseName = in.readUTF(); + this.tableName = in.readUTF(); + } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java new file mode 100644 index 00000000000..5afa381ba9b --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java @@ -0,0 +1,47 @@ +/* + * 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.carbondata.sdk.store.service; + +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.sdk.store.service.model.BaseResponse; +import org.apache.carbondata.sdk.store.service.model.LoadDataRequest; +import org.apache.carbondata.sdk.store.service.model.ScanRequest; +import org.apache.carbondata.sdk.store.service.model.ScanResponse; + +import org.apache.hadoop.ipc.VersionedProtocol; + +public interface DataService extends VersionedProtocol { + + long versionID = 1L; + + /** + * Load data into a Table + * @param load descriptor for load operation + * @throws CarbonException if any error occurs + */ + BaseResponse loadData(LoadDataRequest load) throws CarbonException; + + /** + * Scan a Table and return matched rows + * @param scan descriptor for scan operation, including required column, filter, etc + * @return matched rows + * @throws CarbonException if any error occurs + */ + ScanResponse scan(ScanRequest scan) throws CarbonException; + +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/PruneService.java similarity index 85% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/PruneService.java index daef01bc02b..c1316da853e 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/PruneService.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/PruneService.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc; +package org.apache.carbondata.sdk.store.service; import java.io.IOException; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.store.impl.rpc.model.PruneRequest; -import org.apache.carbondata.store.impl.rpc.model.PruneResponse; +import org.apache.carbondata.sdk.store.service.model.PruneRequest; +import org.apache.carbondata.sdk.store.service.model.PruneResponse; import org.apache.hadoop.ipc.VersionedProtocol; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java new file mode 100644 index 00000000000..e3721d5c571 --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java @@ -0,0 +1,56 @@ +/* + * 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.carbondata.sdk.store.service; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; + +import org.apache.carbondata.common.annotations.InterfaceAudience; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.ipc.RPC; + +@InterfaceAudience.Internal +public class ServiceFactory { + + public static StoreService createStoreService(String host, int port) throws IOException { + InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); + return RPC.getProxy(StoreService.class, StoreService.versionID, address, new Configuration()); + } + + public static PruneService createPruneService(String host, int port) throws IOException { + InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); + return RPC.getProxy( + PruneService.class, PruneService.versionID, address, new Configuration()); + } + + public static DataService createDataService(String host, int port) throws IOException { + InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); + return RPC.getProxy( + DataService.class, DataService.versionID, address, new Configuration()); + } + + public static void stopStoreService(StoreService service) { + RPC.stopProxy(service); + } + + public static void stopDataService(DataService service) { + RPC.stopProxy(service); + } +} \ No newline at end of file diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java new file mode 100644 index 00000000000..fa5f0656c1f --- /dev/null +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java @@ -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.carbondata.sdk.store.service; + +import java.util.List; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +import org.apache.hadoop.ipc.VersionedProtocol; + +@InterfaceAudience.Internal +public interface StoreService extends VersionedProtocol { + long versionID = 1L; + + void createTable(TableDescriptor descriptor) throws CarbonException; + + void dropTable(TableIdentifier table) throws CarbonException; + + CarbonTable getTable(TableIdentifier table) throws CarbonException; + + List listTable() throws CarbonException; + + TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException; + + void alterTable(TableIdentifier table, TableDescriptor newTable) throws CarbonException; + + void loadData(LoadDescriptor loadDescriptor) throws CarbonException; + + List scan(ScanDescriptor scanDescriptor) throws CarbonException; + +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/BaseResponse.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/BaseResponse.java similarity index 97% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/BaseResponse.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/BaseResponse.java index b11aa69c843..b1e87362f9f 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/BaseResponse.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/BaseResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.sdk.store.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/LoadDataRequest.java similarity index 97% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/LoadDataRequest.java index 552c5cb0cb3..0956ef3e0c8 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/LoadDataRequest.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/LoadDataRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.sdk.store.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneRequest.java similarity index 96% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneRequest.java index 0223c380164..61d98260b9c 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneRequest.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.sdk.store.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneResponse.java similarity index 94% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneResponse.java index 9383ac1595c..7b071328ed2 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/PruneResponse.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.sdk.store.service.model; import java.io.DataInput; import java.io.DataOutput; @@ -25,8 +25,8 @@ import java.util.List; import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.sdk.store.BlockScanUnit; import org.apache.carbondata.sdk.store.ScanUnit; -import org.apache.carbondata.store.impl.BlockScanUnit; import org.apache.hadoop.io.Writable; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/Scan.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java similarity index 93% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/Scan.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java index 86767e5c336..209e3f186b8 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/Scan.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.sdk.store.service.model; import java.io.DataInput; import java.io.DataOutput; @@ -31,7 +31,7 @@ import org.apache.hadoop.io.Writable; @InterfaceAudience.Internal -public class Scan implements Serializable, Writable { +public class ScanRequest implements Serializable, Writable { private int requestId; private CarbonMultiBlockSplit split; private TableInfo tableInfo; @@ -39,10 +39,10 @@ public class Scan implements Serializable, Writable { private Expression filterExpression; private long limit; - public Scan() { + public ScanRequest() { } - public Scan(int requestId, CarbonMultiBlockSplit split, + public ScanRequest(int requestId, CarbonMultiBlockSplit split, TableInfo tableInfo, String[] projectColumns, Expression filterExpression, long limit) { this.requestId = requestId; this.split = split; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/QueryResponse.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java similarity index 88% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/QueryResponse.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java index 304fd0fe9c2..eaeafa50a72 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/QueryResponse.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.sdk.store.service.model; import java.io.DataInput; import java.io.DataOutput; @@ -29,15 +29,15 @@ import org.apache.hadoop.io.WritableUtils; @InterfaceAudience.Internal -public class QueryResponse extends BaseResponse implements Serializable, Writable { +public class ScanResponse extends BaseResponse implements Serializable, Writable { private int queryId; private Object[][] rows; - public QueryResponse() { + public ScanResponse() { super(); } - public QueryResponse(int queryId, int status, String message, Object[][] rows) { + public ScanResponse(int queryId, int status, String message, Object[][] rows) { super(status, message); this.queryId = queryId; this.rows = rows; From f57646d1e26e7380c01f05fd8e524d8ecec6cdfb Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Fri, 3 Aug 2018 12:04:47 +0800 Subject: [PATCH 08/14] remove getTable API in CarbonStore --- .../core/metadata/schema/table/TableInfo.java | 2 +- .../store/impl/LocalCarbonStore.java | 9 +++++--- .../carbondata/store/impl/TableManager.java | 9 ++++---- .../store/impl/master/StoreServiceImpl.java | 3 ++- ...t.java => DistributedCarbonStoreTest.java} | 6 +++--- .../carbondata/sdk/store/CarbonStore.java | 18 ++++++---------- .../sdk/store/CarbonStoreFactory.java | 2 +- ...Store.java => DistributedCarbonStore.java} | 21 +++++++------------ .../sdk/store/service/StoreService.java | 3 ++- 9 files changed, 33 insertions(+), 40 deletions(-) rename store/core/src/test/java/org/apache/carbondata/store/{RemoteCarbonStoreTest.java => DistributedCarbonStoreTest.java} (95%) rename store/sdk/src/main/java/org/apache/carbondata/sdk/store/{RemoteCarbonStore.java => DistributedCarbonStore.java} (83%) diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java index 46328f714f6..abe1810d664 100644 --- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java +++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/TableInfo.java @@ -44,7 +44,7 @@ * Store the information about the table. * it stores the fact table as well as aggregate table present in the schema */ -public class TableInfo implements Serializable, Writable { +public class TableInfo implements Serializable, Writable, org.apache.hadoop.io.Writable { private static final LogService LOGGER = LogServiceFactory.getLogService(TableInfo.class.getName()); diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index c825bea58ad..bf0844b265b 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -31,6 +31,7 @@ import org.apache.carbondata.core.datastore.block.Distributable; import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.core.mutate.CarbonUpdateUtil; import org.apache.carbondata.core.util.ThreadLocalTaskInfo; import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; @@ -92,7 +93,8 @@ public void loadData(LoadDescriptor load) throws CarbonException { Objects.requireNonNull(load); CarbonLoadModel loadModel; try { - CarbonTable table = getTable(load.getTable()); + TableInfo tableInfo = getTable(load.getTable()); + CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo); CarbonLoadModelBuilder modelBuilder = new CarbonLoadModelBuilder(table); modelBuilder.setInputPath(load.getInputPath()); loadModel = modelBuilder.build(load.getOptions(), System.currentTimeMillis(), "0"); @@ -165,11 +167,12 @@ private void loadData(CarbonLoadModel model) throws Exception { public List scan(ScanDescriptor scanDescriptor) throws CarbonException { Objects.requireNonNull(scanDescriptor); try { - CarbonTable table = getTable(scanDescriptor.getTableIdentifier()); + TableInfo tableInfo = getTable(scanDescriptor.getTableIdentifier()); + CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo); List blocks = pruneBlock(table, scanDescriptor.getFilter()); CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, ""); ScanRequest scan = - new ScanRequest(0, split, table.getTableInfo(), scanDescriptor.getProjection(), + new ScanRequest(0, split, tableInfo, scanDescriptor.getProjection(), scanDescriptor.getFilter(), scanDescriptor.getLimit()); return scan(table, scan); } catch (IOException e) { diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java b/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java index 142624c0850..78072d47aec 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java @@ -82,7 +82,7 @@ public class TableManager { private StoreConf storeConf; // mapping of table path to CarbonTable object - private Map cache = new HashMap<>(); + private Map cache = new HashMap<>(); public TableManager(StoreConf storeConf) { this.storeConf = storeConf; @@ -183,7 +183,7 @@ public void dropTable(TableIdentifier table) throws CarbonException { } } - public CarbonTable getTable(TableIdentifier table) throws CarbonException { + public TableInfo getTable(TableIdentifier table) throws CarbonException { String tablePath = getTablePath(table.getTableName(), table.getDatabaseName()); if (cache.containsKey(tablePath)) { return cache.get(tablePath); @@ -198,9 +198,8 @@ public CarbonTable getTable(TableIdentifier table) throws CarbonException { TableInfo tableInfo = schemaConverter.fromExternalToWrapperTableInfo( formatTableInfo, table.getDatabaseName(), table.getTableName(), tablePath); tableInfo.setTablePath(tablePath); - CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo); - cache.put(tablePath, carbonTable); - return carbonTable; + cache.put(tablePath, tableInfo); + return tableInfo; } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java index aceba74e9d1..c0fb8efe19e 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java @@ -22,6 +22,7 @@ import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; @@ -57,7 +58,7 @@ public void dropTable(TableIdentifier identifier) throws CarbonException { } @Override - public CarbonTable getTable(TableIdentifier identifier) throws CarbonException { + public TableInfo getTable(TableIdentifier identifier) throws CarbonException { return tableManager.getTable(identifier); } diff --git a/store/core/src/test/java/org/apache/carbondata/store/RemoteCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java similarity index 95% rename from store/core/src/test/java/org/apache/carbondata/store/RemoteCarbonStoreTest.java rename to store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java index f9c49cb4354..66c64d39272 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/RemoteCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java @@ -45,14 +45,14 @@ import org.junit.BeforeClass; import org.junit.Test; -public class RemoteCarbonStoreTest { +public class DistributedCarbonStoreTest { private static String projectFolder; private static CarbonStore store; @BeforeClass public static void beforeAll() throws IOException, CarbonException, InterruptedException { - projectFolder = new File(RemoteCarbonStoreTest.class.getResource("/").getPath() + + projectFolder = new File(DistributedCarbonStoreTest.class.getResource("/").getPath() + "../../../../").getCanonicalPath(); String confFile = projectFolder + "/store/conf/store.conf"; @@ -73,7 +73,7 @@ public static void beforeAll() throws IOException, CarbonException, InterruptedE Thread.sleep(1000); - store = CarbonStoreFactory.getRemoteStore("RemoteCarbonStoreTest", storeConf); + store = CarbonStoreFactory.getRemoteStore("DistributedCarbonStoreTest", storeConf); } @AfterClass diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java index d666331a868..48d382ced31 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -55,14 +55,6 @@ public interface CarbonStore extends Closeable { */ void dropTable(TableIdentifier table) throws CarbonException; - /** - * Return CarbonTable object by specified identifier - * @param table table identifier - * @return CarbonTable object - * @throws CarbonException if any error occurs - */ - CarbonTable getTable(TableIdentifier table) throws CarbonException; - /** * @return all table created * @throws CarbonException if any error occurs @@ -98,7 +90,7 @@ public interface CarbonStore extends Closeable { void loadData(LoadDescriptor load) throws CarbonException; /** - * Create a new Loader, which can be used to load data in distributed compute framework + * Return a new Loader that can be used to load data in distributed compute framework * @param load descriptor for load operation * @return a new Loader * @throws CarbonException if any error occurs @@ -132,16 +124,18 @@ default Mutator newMutator() throws CarbonException { //////////////////////////////////////////////////////////////////// /** - * Scan a Table and return matched rows, using default select option - * see {@link #scan(ScanDescriptor, SelectOption)} for more information + * Scan the specified table and return matched rows * - * @param select descriptor for select operation + * @param select descriptor for scan operation * @return matched rows * @throws CarbonException if any error occurs */ List scan(ScanDescriptor select) throws CarbonException; /** + * Return a new Scanner that can be used in for parallel scan + * + * @param tableIdentifier table to scan * @return a new Scanner * @throws CarbonException if any error occurs */ diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java index 64019786d36..b09b8af7907 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java @@ -48,7 +48,7 @@ public static CarbonStore getRemoteStore(String storeName, StoreConf storeConf) // create a new instance try { - String className = "org.apache.carbondata.sdk.store.RemoteCarbonStore"; + String className = "org.apache.carbondata.sdk.store.DistributedCarbonStore"; CarbonStore store = createCarbonStore(storeConf, className); remoteStore.put(storeName, store); return store; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java similarity index 83% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java index 64bc7485025..10a2be8aff9 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RemoteCarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java @@ -25,7 +25,7 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; @@ -38,18 +38,18 @@ /** * A CarbonStore that leverage multiple servers via RPC calls (Master and Workers) */ -public class RemoteCarbonStore implements CarbonStore { +public class DistributedCarbonStore implements CarbonStore { private static final LogService LOGGER = - LogServiceFactory.getLogService(RemoteCarbonStore.class.getCanonicalName()); + LogServiceFactory.getLogService(DistributedCarbonStore.class.getCanonicalName()); private static final long versionID = 1L; private StoreService storeService; private StoreConf storeConf; - private Map tableCache = new ConcurrentHashMap<>(); + private Map tableCache = new ConcurrentHashMap<>(); - public RemoteCarbonStore(StoreConf conf) throws IOException { + public DistributedCarbonStore(StoreConf conf) throws IOException { this.storeService = ServiceFactory.createStoreService(conf.masterHost(), conf.storeServicePort()); this.storeConf = conf; @@ -73,11 +73,6 @@ public void dropTable(TableIdentifier table) throws CarbonException { } } - @Override - public CarbonTable getTable(TableIdentifier table) throws CarbonException { - return storeService.getTable(table); - } - @Override public List listTable() throws CarbonException { return storeService.listTable(); @@ -115,10 +110,10 @@ public List scan(ScanDescriptor select) throws CarbonException { @Override public Scanner newScanner(TableIdentifier identifier) throws CarbonException { - CarbonTable table = tableCache.getOrDefault(identifier, storeService.getTable(identifier)); - tableCache.putIfAbsent(identifier, table); + TableInfo tableInfo = tableCache.getOrDefault(identifier, storeService.getTable(identifier)); + tableCache.putIfAbsent(identifier, tableInfo); try { - return new ScannerImpl(storeConf, table.getTableInfo()); + return new ScannerImpl(storeConf, tableInfo); } catch (IOException e) { throw new CarbonException(e); } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java index fa5f0656c1f..b825b07ad0b 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java @@ -22,6 +22,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; @@ -38,7 +39,7 @@ public interface StoreService extends VersionedProtocol { void dropTable(TableIdentifier table) throws CarbonException; - CarbonTable getTable(TableIdentifier table) throws CarbonException; + TableInfo getTable(TableIdentifier table) throws CarbonException; List listTable() throws CarbonException; From f72850a739d18e2bec21672e05b31278e084eb22 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Fri, 3 Aug 2018 12:47:16 +0800 Subject: [PATCH 09/14] fix ScannerImpl serialization --- .../org/apache/carbondata/sdk/store/ScannerImpl.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java index 8c1fd1b25d0..1e5b0f2c705 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java @@ -52,13 +52,14 @@ class ScannerImpl implements Scanner { private static final LogService LOGGER = LogServiceFactory.getLogService(ScannerImpl.class.getCanonicalName()); - private PruneService pruneService; private TableInfo tableInfo; + private String pruneServiceHost; + private int pruneServiePort; ScannerImpl(StoreConf conf, TableInfo tableInfo) throws IOException { - this.pruneService = ServiceFactory.createPruneService( - conf.masterHost(), conf.registryServicePort()); this.tableInfo = tableInfo; + this.pruneServiceHost = conf.masterHost(); + this.pruneServiePort = conf.pruneServicePort(); } /** @@ -77,6 +78,8 @@ public List prune(TableIdentifier table, Expression filterExpression) CarbonInputFormat.setDatabaseName(configuration, table.getDatabaseName()); CarbonInputFormat.setFilterPredicates(configuration, filterExpression); PruneRequest request = new PruneRequest(configuration); + PruneService pruneService = ServiceFactory.createPruneService( + pruneServiceHost, pruneServiePort); PruneResponse response = pruneService.prune(request); return response.getScanUnits(); } catch (IOException e) { From 58866e2d7c80ffebef47a7cb1323b69c49dcb6f5 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 4 Aug 2018 12:45:09 +0800 Subject: [PATCH 10/14] fix comment --- .../carbondata/hadoop/CarbonInputSplit.java | 31 ++++++++++++++- .../store/impl/master/StoreServiceImpl.java | 1 - .../store/DistributedCarbonStoreTest.java | 4 +- .../store/LocalCarbonStoreTest.java | 4 +- .../carbondata/sdk/store/BlockScanUnit.java | 3 ++ .../carbondata/sdk/store/CarbonStore.java | 13 ++++--- .../sdk/store/DistributedCarbonStore.java | 6 ++- .../{ScannerImpl.java => RowScanner.java} | 32 +++++---------- .../{SelectOption.java => ScanOption.java} | 2 +- .../apache/carbondata/sdk/store/ScanUnit.java | 2 +- .../apache/carbondata/sdk/store/Scanner.java | 39 ++++++++++++------- .../carbondata/sdk/store/Schedulable.java | 29 +++++++++++++- .../sdk/store/descriptor/ScanDescriptor.java | 4 +- .../sdk/store/service/StoreService.java | 1 - 14 files changed, 115 insertions(+), 56 deletions(-) rename store/sdk/src/main/java/org/apache/carbondata/sdk/store/{ScannerImpl.java => RowScanner.java} (82%) rename store/sdk/src/main/java/org/apache/carbondata/sdk/store/{SelectOption.java => ScanOption.java} (97%) diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java index f59f72db7b3..7f1c8b11443 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java @@ -42,6 +42,7 @@ import org.apache.carbondata.hadoop.internal.index.Block; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.lib.input.FileSplit; @@ -56,6 +57,11 @@ public class CarbonInputSplit extends FileSplit private Segment segment; + // We use this filePath to store the block location instead of the + // filePath in FileSplit, because filePath in FileSplit is not Serializable + // before Hadoop 3, see HADOOP-13519 + private String filePath; + private String bucketId; private String blockletId; @@ -98,6 +104,7 @@ public CarbonInputSplit() { numberOfBlocklets = 0; invalidSegments = new ArrayList<>(); version = CarbonProperties.getInstance().getFormatVersion(); + filePath = ""; } private CarbonInputSplit(String segmentId, String blockletId, Path path, long start, long length, @@ -116,6 +123,7 @@ private CarbonInputSplit(String segmentId, String blockletId, Path path, long st this.version = version; this.deleteDeltaFiles = deleteDeltaFiles; this.dataMapWritePath = dataMapWritePath; + this.filePath = path.toString(); } public CarbonInputSplit(String segmentId, String blockletId, Path path, long start, long length, @@ -136,6 +144,7 @@ public CarbonInputSplit(String segmentId, Path path, long start, long length, St numberOfBlocklets = 0; invalidSegments = new ArrayList<>(); version = CarbonProperties.getInstance().getFormatVersion(); + filePath = path.toString(); } public CarbonInputSplit(String segmentId, Path path, long start, long length, String[] locations, @@ -149,6 +158,7 @@ public CarbonInputSplit(String segmentId, Path path, long start, long length, St numberOfBlocklets = 0; invalidSegments = new ArrayList<>(); version = CarbonProperties.getInstance().getFormatVersion(); + filePath = path.toString(); } /** @@ -252,9 +262,24 @@ public Segment getSegment() { if (dataMapWriterPathExists) { dataMapWritePath = in.readUTF(); } + boolean filePathExists = in.readBoolean(); + if (filePathExists) { + filePath = in.readUTF(); + } else { + filePath = super.getPath().toString(); + } } @Override public void write(DataOutput out) throws IOException { + if (super.getPath() != null) { + super.write(out); + } else { + // see HADOOP-13519, after Java deserialization, super.filePath is + // null, so write our filePath instead + Text.writeString(out, filePath); + out.writeLong(getStart()); + out.writeLong(getLength()); + } super.write(out); out.writeUTF(segment.toString()); out.writeShort(version.number()); @@ -278,6 +303,10 @@ public Segment getSegment() { if (dataMapWritePath != null) { out.writeUTF(dataMapWritePath); } + out.writeBoolean(filePath != null); + if (filePath != null) { + out.writeUTF(filePath); + } } public List getInvalidSegments() { @@ -398,7 +427,7 @@ public String getBucketId() { } @Override public String getBlockPath() { - return getPath().getName(); + return filePath.substring(filePath.lastIndexOf("/") + 1); } @Override public List getMatchedBlocklets() { diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java index c0fb8efe19e..0c90770d9e4 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; diff --git a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java index 66c64d39272..0677f1141ca 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java @@ -130,7 +130,7 @@ public void testSelect() throws CarbonException { ScanDescriptor select = ScanDescriptor .builder() .table(tableIdentifier) - .select("intField", "stringField") + .select(new String[]{"intField", "stringField"}) .limit(5) .create(); List result = store.scan(select); @@ -140,7 +140,7 @@ public void testSelect() throws CarbonException { ScanDescriptor select2 = ScanDescriptor .builder() .table(tableIdentifier) - .select("intField", "stringField") + .select(new String[]{"intField", "stringField"}) .filter(new EqualToExpression( new ColumnExpression("intField", DataTypes.INT), new LiteralExpression(11, DataTypes.INT))) diff --git a/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java index 09a326280ef..8cf7e1402a3 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/LocalCarbonStoreTest.java @@ -109,7 +109,7 @@ public void testWriteAndReadFiles() throws IOException, CarbonException { ScanDescriptor select = ScanDescriptor .builder() .table(tableIdentifier) - .select("intField", "stringField") + .select(new String[]{"intField", "stringField"}) .limit(5) .create(); List result = store.scan(select); @@ -119,7 +119,7 @@ public void testWriteAndReadFiles() throws IOException, CarbonException { ScanDescriptor select2 = ScanDescriptor .builder() .table(tableIdentifier) - .select("intField", "stringField") + .select(new String[]{"intField", "stringField"}) .filter(new EqualToExpression( new ColumnExpression("intField", DataTypes.INT), new LiteralExpression(11, DataTypes.INT))) diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java index f795bff537a..283148ca901 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java @@ -60,11 +60,14 @@ public String[] preferredLocations() { @Override public void write(DataOutput out) throws IOException { inputSplit.write(out); + schedulable.write(out); } @Override public void readFields(DataInput in) throws IOException { inputSplit = new CarbonInputSplit(); inputSplit.readFields(in); + schedulable = new Schedulable(); + schedulable.readFields(in); } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java index 48d382ced31..9f390c80b6b 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -23,7 +23,6 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; @@ -101,7 +100,8 @@ public interface CarbonStore extends Closeable { * Return true if this table has primary key defined when create table using * {@link #createTable(TableDescriptor)} * - * For a such table, {@link #newMutator()} and {@link #newFetcher()} are supported + * For such table, {@link #newMutator(TableIdentifier)} and {@link #newFetcher(TableIdentifier)} + * are supported * * @return true if this table has primary key. */ @@ -110,11 +110,12 @@ default boolean isPrimaryKeyDefined() { } /** - * A mutator supports upsert and delete using primary key + * Return a new mutator that supports realtime insert, upsert and delete using primary key + * * @return a new mutator * @throws CarbonException if any error occurs */ - default Mutator newMutator() throws CarbonException { + default Mutator newMutator(TableIdentifier tableIdentifier) throws CarbonException { throw new UnsupportedOperationException(); } @@ -142,10 +143,12 @@ default Mutator newMutator() throws CarbonException { Scanner newScanner(TableIdentifier tableIdentifier) throws CarbonException; /** + * Return a new Fetch that can be used for lookup operation + * * @return a new Fetcher * @throws CarbonException if any error occurs */ - default Fetcher newFetcher() throws CarbonException { + default Fetcher newFetcher(TableIdentifier tableIdentifier) throws CarbonException { throw new UnsupportedOperationException(); } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java index 10a2be8aff9..806348cf94b 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java @@ -80,7 +80,9 @@ public List listTable() throws CarbonException { @Override public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { - return storeService.getDescriptor(table); + TableInfo tableInfo = storeService.getTable(table); + // TODO: create TableDescriptor from tableInfo + return null; } @Override @@ -113,7 +115,7 @@ public Scanner newScanner(TableIdentifier identifier) throws CarbonException { TableInfo tableInfo = tableCache.getOrDefault(identifier, storeService.getTable(identifier)); tableCache.putIfAbsent(identifier, tableInfo); try { - return new ScannerImpl(storeConf, tableInfo); + return new RowScanner(storeConf, tableInfo); } catch (IOException e) { throw new CarbonException(e); } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowScanner.java similarity index 82% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowScanner.java index 1e5b0f2c705..914238606fe 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScannerImpl.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowScanner.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.stream.Collectors; @@ -48,27 +49,20 @@ import org.apache.hadoop.conf.Configuration; -class ScannerImpl implements Scanner { +class RowScanner implements Scanner { private static final LogService LOGGER = - LogServiceFactory.getLogService(ScannerImpl.class.getCanonicalName()); + LogServiceFactory.getLogService(RowScanner.class.getCanonicalName()); private TableInfo tableInfo; private String pruneServiceHost; private int pruneServiePort; - ScannerImpl(StoreConf conf, TableInfo tableInfo) throws IOException { + RowScanner(StoreConf conf, TableInfo tableInfo) throws IOException { this.tableInfo = tableInfo; this.pruneServiceHost = conf.masterHost(); this.pruneServiePort = conf.pruneServicePort(); } - /** - * Trigger a RPC to Carbon Master to do pruning - * @param table table identifier - * @param filterExpression expression of filter predicate given by user - * @return list of ScanUnit - * @throws CarbonException if any error occurs - */ @Override public List prune(TableIdentifier table, Expression filterExpression) throws CarbonException { @@ -87,17 +81,11 @@ public List prune(TableIdentifier table, Expression filterExpression) } } - /** - * Execute scan by firing RPC call to worker, return the result rows - * @param input one scan unit - * @param descriptor parameter for scan - * @param option scan options - * @return result batch - * @throws CarbonException if any error occurs - */ @Override - public Iterator> scan(ScanUnit input, ScanDescriptor descriptor, - SelectOption option) throws CarbonException { + public Iterator> scan( + ScanUnit input, + ScanDescriptor scanDescriptor, + Map option) throws CarbonException { List toBeScan = new ArrayList<>(); if (input instanceof BlockScanUnit) { toBeScan.add(((BlockScanUnit) input).getInputSplit()); @@ -107,8 +95,8 @@ public Iterator> scan(ScanUnit input, ScanDescr int queryId = new Random().nextInt(); CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(toBeScan, input.preferredLocations()); try { - ScanRequest request = new ScanRequest(queryId, split, tableInfo, descriptor.getProjection(), - descriptor.getFilter(), descriptor.getLimit()); + ScanRequest request = new ScanRequest(queryId, split, tableInfo, + scanDescriptor.getProjection(), scanDescriptor.getFilter(), scanDescriptor.getLimit()); DataService dataService = DataServicePool.getOrCreateDataService(((BlockScanUnit) input).getSchedulable()); ScanResponse response = dataService.scan(request); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanOption.java similarity index 97% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java rename to store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanOption.java index bafa26ea1c7..cfba174c93e 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SelectOption.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanOption.java @@ -22,7 +22,7 @@ @InterfaceAudience.User @InterfaceStability.Unstable -public class SelectOption { +public class ScanOption { public static final String BATCH_SIZE = "batchSize"; public static final String ENABLE_ROW_MAJOR = "rowMajor"; } \ No newline at end of file diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java index a5d869cbe69..a825f208e4c 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java @@ -28,7 +28,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface ScanUnit extends Serializable, Writable { +public interface ScanUnit extends Serializable, Writable { /** * Return the list of preferred location of this ScanUnit. diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java index 366a7d3a023..a8597e713a6 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java @@ -20,53 +20,62 @@ import java.io.Serializable; import java.util.Iterator; import java.util.List; +import java.util.Map; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; /** - * A Scanner is used to scan the table + * A Scanner is used to scan the table in a distributed compute + * engine like Apache Spark */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface Scanner extends Serializable { +public interface Scanner extends Serializable { /** * Return an array of ScanUnit which will be the input in - * {@link #scan(ScanUnit, ScanDescriptor, SelectOption)} + * {@link #scan(ScanUnit, ScanDescriptor, Map)} * - * Implementation will leverage index to prune using specified filter expression + * Implementation will leverage index to prune using specified + * filter expression * * @param table table identifier * @param filterExpression expression of filter predicate given by user * @return list of ScanUnit which should be passed to - * {@link #scan(ScanUnit, ScanDescriptor, SelectOption)} + * {@link #scan(ScanUnit, ScanDescriptor, Map)} * @throws CarbonException if any error occurs */ - List prune(TableIdentifier table, Expression filterExpression) throws CarbonException; + List prune( + TableIdentifier table, + Expression filterExpression) throws CarbonException; /** * Perform a scan in a distributed compute framework like Spark, Presto, etc. * Filter/Projection/Limit operation is pushed down to the scan. * - * This should be used with {@link #prune(TableIdentifier, Expression)} in a distributed - * compute environment. It enables the framework to do a parallel scan by creating - * multiple {@link ScanUnit} and perform parallel scan in worker, such as Spark executor + * This should be used with {@link #prune(TableIdentifier, Expression)} + * in a distributed compute environment. It enables the framework to + * do a parallel scan by creating multiple {@link ScanUnit} and perform + * parallel scan in worker, such as Spark executor * - * The return result is in batch so that the caller can start next level of computation - * before getting all results, such as implementing a `prefetch` execution model. + * The return result is in batch so that the caller can start next + * level of computation before getting all results, such as + * implementing a `prefetch` execution model. * * @param input one scan unit - * @param select parameter for scanning + * @param scanDescriptor parameter for scanning + * @param scanOption options for scan, use {@link ScanOption} for the map key * @return scan result, the result is returned in batch * @throws CarbonException if any error occurs */ - Iterator> scan(ScanUnit input, ScanDescriptor select, - SelectOption option) throws CarbonException; + Iterator> scan( + ScanUnit input, + ScanDescriptor scanDescriptor, + Map scanOption) throws CarbonException; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java index 95cf3d047a6..827816d9711 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java @@ -17,12 +17,17 @@ package org.apache.carbondata.sdk.store; +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; import java.util.concurrent.atomic.AtomicInteger; import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.hadoop.io.Writable; + @InterfaceAudience.Internal -public class Schedulable { +public class Schedulable implements Writable { private String id; private String address; @@ -30,6 +35,9 @@ public class Schedulable { private int cores; public AtomicInteger workload; + public Schedulable() { + } + public Schedulable(String id, String address, int port, int cores) { this.id = id; this.address = address; @@ -70,4 +78,23 @@ public int getCores() { return "Schedulable{" + "id='" + id + '\'' + ", address='" + address + '\'' + ", port=" + port + '}'; } + + @Override + public void write(DataOutput out) throws IOException { + out.writeUTF(id); + out.writeUTF(address); + out.writeInt(port); + out.writeInt(cores); + // We are not writing workload since it is only useful for + // Scheduler inside the Master. Client of the Master does + // not need it + } + + @Override + public void readFields(DataInput in) throws IOException { + id = in.readUTF(); + address = in.readUTF(); + port = in.readInt(); + cores = in.readInt(); + } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java index f863f1db3b2..1b8ea53736b 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java @@ -36,7 +36,7 @@ public class ScanDescriptor implements Writable { private TableIdentifier table; private String[] projection; private Expression filter; - private long limit; + private long limit = Long.MAX_VALUE; private ScanDescriptor() { } @@ -124,7 +124,7 @@ public Builder table(TableIdentifier tableIdentifier) { return this; } - public Builder select(String... columnNames) { + public Builder select(String[] columnNames) { select.setProjection(columnNames); return this; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java index b825b07ad0b..7e54d2f5548 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java @@ -21,7 +21,6 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; From 9b081faeb001046b1be5a49501b832a2ddd56ea1 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 4 Aug 2018 16:06:58 +0800 Subject: [PATCH 11/14] add InternalCarbonStore --- .../org/apache/spark/sql/CarbonSession.scala | 2 +- .../zeppelin/TestCarbonResponse.java | 4 +- .../store/devapi/InternalCarbonStore.java | 65 +++++++++++++++++++ .../devapi/InternalCarbonStoreFactory.java} | 18 ++--- .../store/devapi/InternalCarbonStoreImpl.java | 63 ++++++++++++++++++ .../carbondata/store/devapi}/Loader.java | 3 +- .../carbondata/store/devapi}/ResultBatch.java | 2 +- .../carbondata/store/devapi}/ScanOption.java | 2 +- .../carbondata/store/devapi}/ScanUnit.java | 2 +- .../carbondata/store/devapi}/Scanner.java | 2 +- .../store/devapi}/TransactionalOperation.java | 2 +- .../carbondata/store/impl}/BlockScanUnit.java | 3 +- .../store/impl}/DataServicePool.java | 11 ++-- .../store/impl/IndexedRecordReader.java | 2 +- .../store/impl/LocalCarbonStore.java | 24 +++++-- .../store/impl}/RowMajorResultBatch.java | 12 ++-- .../carbondata/store/impl}/RowScanner.java | 38 ++++++----- .../carbondata/store/impl}/Schedulable.java | 2 +- .../carbondata/store/impl/TableManager.java | 2 +- .../carbondata/store/impl/master/Master.java | 10 +-- .../store/impl/master/PruneServiceImpl.java | 12 ++-- .../impl/master/RegistryServiceImpl.java | 6 +- .../store/impl/master/Scheduler.java | 2 +- .../store/impl/master/StoreServiceImpl.java | 6 +- .../store/impl}/service/DataService.java | 10 +-- .../store/impl}/service/PruneService.java | 6 +- .../{rpc => service}/RegistryService.java | 6 +- .../impl/{rpc => service}/ServiceFactory.java | 14 +++- .../impl}/service/model/BaseResponse.java | 2 +- .../impl}/service/model/LoadDataRequest.java | 2 +- .../impl}/service/model/PruneRequest.java | 2 +- .../impl}/service/model/PruneResponse.java | 6 +- .../model/RegisterWorkerRequest.java | 2 +- .../model/RegisterWorkerResponse.java | 2 +- .../impl}/service/model/ScanRequest.java | 2 +- .../impl}/service/model/ScanResponse.java | 2 +- .../model/ShutdownRequest.java | 2 +- .../model/ShutdownResponse.java | 2 +- .../store/impl/worker/DataRequestHandler.java | 12 ++-- .../store/impl/worker/DataServiceImpl.java | 10 +-- .../carbondata/store/impl/worker/Worker.java | 10 +-- .../store/DistributedCarbonStoreTest.java | 2 +- .../horizon/rest/client/HorizonClient.java | 10 +-- .../rest/client/impl/SimpleHorizonClient.java | 12 ++-- .../horizon/rest/controller/Horizon.java | 3 +- .../rest/controller/HorizonController.java | 44 ++++++++----- .../rest/model/validate/RequestValidator.java | 38 +++++------ .../horizon/rest/model/view/FieldRequest.java | 2 +- .../rest/model/view/SelectResponse.java | 11 ++-- .../carbondata/horizon/HorizonTest.java | 8 +-- .../carbondata/sdk/store/CarbonStore.java | 46 +++++++------ .../sdk/store/CarbonStoreFactory.java | 2 +- .../sdk/store/DistributedCarbonStore.java | 42 ++++++------ .../apache/carbondata/sdk/store/Fetcher.java | 37 ----------- .../apache/carbondata/sdk/store/Mutator.java | 48 -------------- .../sdk/store/service/ServiceFactory.java | 15 ----- .../sdk/store/service/StoreService.java | 4 +- .../sdk/file/AvroCarbonWriterTest.java | 4 +- .../carbondata/sdk/file/CarbonReaderTest.java | 24 +++++-- .../rest/controller/SqlHorizonController.java | 8 +-- .../rest/model/validate/RequestValidator.java | 8 +-- 61 files changed, 415 insertions(+), 340 deletions(-) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java => core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java} (68%) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/devapi}/Loader.java (95%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/devapi}/ResultBatch.java (97%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/devapi}/ScanOption.java (96%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/devapi}/ScanUnit.java (97%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/devapi}/Scanner.java (98%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/devapi}/TransactionalOperation.java (96%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/BlockScanUnit.java (95%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/DataServicePool.java (80%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/RowMajorResultBatch.java (80%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/RowScanner.java (74%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/Schedulable.java (98%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/DataService.java (82%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/PruneService.java (85%) rename store/core/src/main/java/org/apache/carbondata/store/impl/{rpc => service}/RegistryService.java (84%) rename store/core/src/main/java/org/apache/carbondata/store/impl/{rpc => service}/ServiceFactory.java (68%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/model/BaseResponse.java (97%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/model/LoadDataRequest.java (97%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/model/PruneRequest.java (96%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/model/PruneResponse.java (92%) rename store/core/src/main/java/org/apache/carbondata/store/impl/{rpc => service}/model/RegisterWorkerRequest.java (97%) rename store/core/src/main/java/org/apache/carbondata/store/impl/{rpc => service}/model/RegisterWorkerResponse.java (96%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/model/ScanRequest.java (98%) rename store/{sdk/src/main/java/org/apache/carbondata/sdk/store => core/src/main/java/org/apache/carbondata/store/impl}/service/model/ScanResponse.java (97%) rename store/core/src/main/java/org/apache/carbondata/store/impl/{rpc => service}/model/ShutdownRequest.java (96%) rename store/core/src/main/java/org/apache/carbondata/store/impl/{rpc => service}/model/ShutdownResponse.java (96%) delete mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java delete mode 100644 store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala index a1687e8369a..3fccfef7809 100644 --- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala +++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonSession.scala @@ -205,7 +205,7 @@ class CarbonSession(@transient val sc: SparkContext, storeConf.conf(StoreConf.WORKER_PORT, CarbonProperties.getSearchWorkerPort) storeConf.conf(StoreConf.WORKER_CORE_NUM, 2) - store = CarbonStoreFactory.getRemoteStore("GlobalStore", storeConf) + store = CarbonStoreFactory.getDistributedStore("GlobalStore", storeConf) CarbonProperties.enableSearchMode(true) CarbonProperties.getInstance() .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER, "false") diff --git a/integration/zeppelin/src/test/java/org/apache/carbondata/zeppelin/TestCarbonResponse.java b/integration/zeppelin/src/test/java/org/apache/carbondata/zeppelin/TestCarbonResponse.java index 4b169f48e13..1f1ce65d7a2 100644 --- a/integration/zeppelin/src/test/java/org/apache/carbondata/zeppelin/TestCarbonResponse.java +++ b/integration/zeppelin/src/test/java/org/apache/carbondata/zeppelin/TestCarbonResponse.java @@ -76,7 +76,7 @@ public void testErrorResponse() throws IOException { " \"timestamp\": 1531884083849,\n" + " \"status\": 500,\n" + " \"error\": \"Internal Server Error\",\n" + - " \"exception\": \"org.apache.carbondata.store.api.exception.StoreException\",\n" + + " \"exception\": \"org.apache.carbondata.store.api.exception.CarbonException\",\n" + " \"message\": \"org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException: " + "Table or view 'sinka6' already exists in database 'default';\",\n" + " \"path\": \"/table/sql\"\n" + @@ -84,7 +84,7 @@ public void testErrorResponse() throws IOException { CarbonResponse errorResponse = CarbonResponse.parse(new ByteArrayInputStream(input.getBytes())).get(); assertEquals("org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException: " + "Table or view 'sinka6' already exists in database 'default';", errorResponse.getMessage()); - assertEquals("org.apache.carbondata.store.api.exception.StoreException", errorResponse.getException()); + assertEquals("org.apache.carbondata.store.api.exception.CarbonException", errorResponse.getException()); assertEquals(1531884083849L, errorResponse.getTimestamp()); assertEquals("Internal Server Error", errorResponse.getError()); assertEquals(500, errorResponse.getStatus()); diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java new file mode 100644 index 00000000000..1c1b5d94203 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java @@ -0,0 +1,65 @@ +/* + * 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.carbondata.store.devapi; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; +import org.apache.carbondata.sdk.store.CarbonStore; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +/** + * Internal API for engine integration developers + */ +@InterfaceAudience.Developer("Integration") +@InterfaceStability.Unstable +public interface InternalCarbonStore extends CarbonStore { + + /** + * Get CarbonTable object from the store + * + * @param tableIdentifier table identifier + * @return CarbonTable object + * @throws CarbonException if any error occurs + */ + CarbonTable getCarbonTable(TableIdentifier tableIdentifier) throws CarbonException; + + /** + * Return a new Loader that can be used to load data in distributed compute framework + * @param load descriptor for load operation + * @return a new Loader + * @throws CarbonException if any error occurs + */ + Loader newLoader(LoadDescriptor load) throws CarbonException; + + /** + * Return a new Scanner that can be used in for parallel scan + * + * @param tableIdentifier table to scan + * @param readSupport read support to convert the row to output object + * @param the target object type contain in {@link ResultBatch} + * @return a new Scanner + * @throws CarbonException if any error occurs + */ + Scanner newRowScanner(TableIdentifier tableIdentifier, CarbonReadSupport readSupport) + throws CarbonException; + +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java similarity index 68% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java index d8e4f83061b..f729064e7df 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/SqlOperation.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java @@ -15,20 +15,16 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import java.io.IOException; -import java.util.List; -import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.sdk.store.conf.StoreConf; -public interface SqlOperation { +public class InternalCarbonStoreFactory { + + public static InternalCarbonStore getStore(StoreConf conf) throws IOException { + return new InternalCarbonStoreImpl(conf); + } - /** - * Executor a SQL statement - * @param sqlString SQL statement - * @return matched rows - * @throws IOException if network or disk IO error occurs - */ - List sql(String sqlString) throws IOException; } diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java new file mode 100644 index 00000000000..0d172179ce9 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java @@ -0,0 +1,63 @@ +/* + * 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.carbondata.store.devapi; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; +import org.apache.carbondata.sdk.store.DistributedCarbonStore; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.impl.RowScanner; + +public class InternalCarbonStoreImpl extends DistributedCarbonStore implements InternalCarbonStore { + + private Map tableCache = new ConcurrentHashMap<>(); + private StoreConf storeConf; + + InternalCarbonStoreImpl(StoreConf storeConf) throws IOException { + super(storeConf); + this.storeConf = storeConf; + } + + @Override + public CarbonTable getCarbonTable(TableIdentifier tableIdentifier) + throws CarbonException { + CarbonTable carbonTable = tableCache.getOrDefault( + tableIdentifier, storeService.getTable(tableIdentifier)); + tableCache.putIfAbsent(tableIdentifier, carbonTable); + return carbonTable; + } + + @Override + public Loader newLoader(LoadDescriptor load) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public Scanner newRowScanner(TableIdentifier identifier, CarbonReadSupport readSupport) + throws CarbonException { + return new RowScanner<>(storeConf, getCarbonTable(identifier), readSupport); + } + +} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/Loader.java similarity index 95% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/Loader.java index cd8f98c1635..d34fa2d7c93 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Loader.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/Loader.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import java.io.Serializable; import java.util.Iterator; @@ -23,6 +23,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.metadata.datatype.StructType; +import org.apache.carbondata.sdk.store.Row; import org.apache.carbondata.sdk.store.exception.CarbonException; /** diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java similarity index 97% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java index 4eccb6aa878..65da069c9b3 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ResultBatch.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanOption.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java similarity index 96% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanOption.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java index cfba174c93e..fdd4ec78d2d 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanOption.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java similarity index 97% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java index a825f208e4c..65db868e1fd 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/ScanUnit.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import java.io.Serializable; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java similarity index 98% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java index a8597e713a6..ab1c9e2dc64 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Scanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import java.io.Serializable; import java.util.Iterator; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java similarity index 96% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java index c55856cad55..9c7249b009b 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/TransactionalOperation.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.devapi; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java b/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java similarity index 95% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java index 283148ca901..d7c219427fa 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/BlockScanUnit.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/BlockScanUnit.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; import java.io.DataInput; import java.io.DataOutput; @@ -23,6 +23,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.store.devapi.ScanUnit; /** * It contains a block to scan, and a destination worker who should scan it diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DataServicePool.java similarity index 80% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/DataServicePool.java index 8f0b4edfa46..8d99c3ed61a 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DataServicePool.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DataServicePool.java @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import org.apache.carbondata.sdk.store.service.DataService; -import org.apache.carbondata.sdk.store.service.ServiceFactory; +import org.apache.carbondata.store.impl.service.DataService; +import org.apache.carbondata.store.impl.service.ServiceFactory; -public class DataServicePool { +class DataServicePool { private static final Map dataServicePool = new ConcurrentHashMap<>(); private DataServicePool() { @@ -38,7 +38,4 @@ static DataService getOrCreateDataService(Schedulable schedulable) throws IOExce return service; } - public static void releaseAll() { - dataServicePool.forEach((schedulable, service) -> ServiceFactory.stopDataService(service)); - } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/IndexedRecordReader.java b/store/core/src/main/java/org/apache/carbondata/store/impl/IndexedRecordReader.java index 64f0742a305..c856d18d83c 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/IndexedRecordReader.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/IndexedRecordReader.java @@ -65,7 +65,7 @@ class IndexedRecordReader extends CarbonRecordReader { private int queryId; private CarbonTable table; - public IndexedRecordReader(int queryId, CarbonTable table, QueryModel queryModel) { + IndexedRecordReader(int queryId, CarbonTable table, QueryModel queryModel) { super(queryModel, new CarbonRowReadSupport()); this.queryId = queryId; this.table = table; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index bf0844b265b..b55e70b6a85 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Date; +import java.util.Iterator; import java.util.List; import java.util.Objects; @@ -30,6 +31,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.block.Distributable; import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.datatype.StructType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.core.mutate.CarbonUpdateUtil; @@ -42,15 +44,16 @@ import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.processing.loading.model.CarbonLoadModelBuilder; import org.apache.carbondata.sdk.store.CarbonStore; -import org.apache.carbondata.sdk.store.Loader; -import org.apache.carbondata.sdk.store.Scanner; +import org.apache.carbondata.sdk.store.KeyedRow; +import org.apache.carbondata.sdk.store.PrimaryKey; +import org.apache.carbondata.sdk.store.Row; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.sdk.store.service.model.ScanRequest; import org.apache.carbondata.sdk.store.util.StoreUtil; +import org.apache.carbondata.store.impl.service.model.ScanRequest; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; @@ -126,7 +129,12 @@ public void loadData(LoadDescriptor load) throws CarbonException { } @Override - public Loader newLoader(LoadDescriptor load) throws CarbonException { + public void upsert(Iterator row, StructType schema) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public void delete(Iterator keys) throws CarbonException { throw new UnsupportedOperationException(); } @@ -181,7 +189,13 @@ public List scan(ScanDescriptor scanDescriptor) throws CarbonExceptio } @Override - public Scanner newScanner(TableIdentifier tableIdentifier) throws CarbonException { + public Row lookup(PrimaryKey key) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public List lookup(TableIdentifier tableIdentifier, String filterExpression) + throws CarbonException { throw new UnsupportedOperationException(); } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowMajorResultBatch.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java similarity index 80% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowMajorResultBatch.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java index 534750ae7b6..1999a63d414 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowMajorResultBatch.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RowMajorResultBatch.java @@ -15,19 +15,19 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; import java.util.Iterator; import java.util.List; import java.util.Objects; -import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.store.devapi.ResultBatch; -public class RowMajorResultBatch implements ResultBatch { +public class RowMajorResultBatch implements ResultBatch { - private Iterator iterator; + private Iterator iterator; - RowMajorResultBatch(List rows) { + RowMajorResultBatch(List rows) { Objects.requireNonNull(rows); this.iterator = rows.iterator(); } @@ -43,7 +43,7 @@ public boolean hasNext() { } @Override - public CarbonRow next() { + public T next() { return iterator.next(); } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RowScanner.java similarity index 74% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowScanner.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/RowScanner.java index 914238606fe..692ba9f7919 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/RowScanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RowScanner.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; import java.io.IOException; import java.util.ArrayList; @@ -29,38 +29,44 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.hadoop.CarbonInputSplit; import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; import org.apache.carbondata.hadoop.api.CarbonInputFormat; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.sdk.store.service.DataService; -import org.apache.carbondata.sdk.store.service.PruneService; -import org.apache.carbondata.sdk.store.service.ServiceFactory; -import org.apache.carbondata.sdk.store.service.model.PruneRequest; -import org.apache.carbondata.sdk.store.service.model.PruneResponse; -import org.apache.carbondata.sdk.store.service.model.ScanRequest; -import org.apache.carbondata.sdk.store.service.model.ScanResponse; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.devapi.Scanner; +import org.apache.carbondata.store.impl.service.DataService; +import org.apache.carbondata.store.impl.service.PruneService; +import org.apache.carbondata.store.impl.service.ServiceFactory; +import org.apache.carbondata.store.impl.service.model.PruneRequest; +import org.apache.carbondata.store.impl.service.model.PruneResponse; +import org.apache.carbondata.store.impl.service.model.ScanRequest; +import org.apache.carbondata.store.impl.service.model.ScanResponse; import org.apache.hadoop.conf.Configuration; -class RowScanner implements Scanner { +public class RowScanner implements Scanner { private static final LogService LOGGER = LogServiceFactory.getLogService(RowScanner.class.getCanonicalName()); private TableInfo tableInfo; private String pruneServiceHost; private int pruneServiePort; + private CarbonReadSupport readSupport; - RowScanner(StoreConf conf, TableInfo tableInfo) throws IOException { - this.tableInfo = tableInfo; + public RowScanner(StoreConf conf, CarbonTable carbonTable, CarbonReadSupport readSupport) { + this.tableInfo = carbonTable.getTableInfo(); this.pruneServiceHost = conf.masterHost(); this.pruneServiePort = conf.pruneServicePort(); + this.readSupport = readSupport; } @Override @@ -82,7 +88,7 @@ public List prune(TableIdentifier table, Expression filterExpression) } @Override - public Iterator> scan( + public Iterator> scan( ScanUnit input, ScanDescriptor scanDescriptor, Map option) throws CarbonException { @@ -100,11 +106,11 @@ public Iterator> scan( DataService dataService = DataServicePool.getOrCreateDataService(((BlockScanUnit) input).getSchedulable()); ScanResponse response = dataService.scan(request); - List rows = Arrays.stream(response.getRows()) - .map(CarbonRow::new) + List rows = Arrays.stream(response.getRows()) + .map(row -> readSupport.readRow(row)) .collect(Collectors.toList()); - return Collections.singletonList(new RowMajorResultBatch(rows)).iterator(); + return Collections.singletonList(new RowMajorResultBatch(rows)).iterator(); } catch (IOException e) { throw new CarbonException(e); } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java b/store/core/src/main/java/org/apache/carbondata/store/impl/Schedulable.java similarity index 98% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/Schedulable.java index 827816d9711..6e29b4861a9 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Schedulable.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/Schedulable.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store; +package org.apache.carbondata.store.impl; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java b/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java index 78072d47aec..92a79caff38 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java @@ -63,7 +63,7 @@ import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.sdk.store.service.model.ScanRequest; +import org.apache.carbondata.store.impl.service.model.ScanRequest; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java index 3e44f78d199..214c89aab1b 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java @@ -28,14 +28,14 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.sdk.store.Schedulable; import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.service.PruneService; import org.apache.carbondata.sdk.store.service.StoreService; import org.apache.carbondata.sdk.store.util.StoreUtil; -import org.apache.carbondata.store.impl.rpc.RegistryService; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; +import org.apache.carbondata.store.impl.Schedulable; +import org.apache.carbondata.store.impl.service.PruneService; +import org.apache.carbondata.store.impl.service.RegistryService; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerRequest; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.RPC; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java index b56b0ae3491..f9f8af95acf 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java @@ -24,12 +24,12 @@ import org.apache.carbondata.hadoop.CarbonInputSplit; import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; -import org.apache.carbondata.sdk.store.BlockScanUnit; -import org.apache.carbondata.sdk.store.ScanUnit; -import org.apache.carbondata.sdk.store.Schedulable; -import org.apache.carbondata.sdk.store.service.PruneService; -import org.apache.carbondata.sdk.store.service.model.PruneRequest; -import org.apache.carbondata.sdk.store.service.model.PruneResponse; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.impl.BlockScanUnit; +import org.apache.carbondata.store.impl.Schedulable; +import org.apache.carbondata.store.impl.service.PruneService; +import org.apache.carbondata.store.impl.service.model.PruneRequest; +import org.apache.carbondata.store.impl.service.model.PruneResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtocolSignature; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java index 492006bc804..73520f6f060 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java @@ -20,9 +20,9 @@ import java.io.IOException; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.store.impl.rpc.RegistryService; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; +import org.apache.carbondata.store.impl.service.RegistryService; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerRequest; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerResponse; import org.apache.hadoop.ipc.ProtocolSignature; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java index 5f9cec7e567..4ede6708a0b 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Scheduler.java @@ -24,8 +24,8 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.util.CarbonProperties; -import org.apache.carbondata.sdk.store.Schedulable; import org.apache.carbondata.sdk.store.exception.SchedulerException; +import org.apache.carbondata.store.impl.Schedulable; /** * [[Master]] uses Scheduler to pick a Worker to send request diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java index 0c90770d9e4..06798265636 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java @@ -21,6 +21,7 @@ import java.util.List; import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; @@ -57,8 +58,9 @@ public void dropTable(TableIdentifier identifier) throws CarbonException { } @Override - public TableInfo getTable(TableIdentifier identifier) throws CarbonException { - return tableManager.getTable(identifier); + public CarbonTable getTable(TableIdentifier identifier) throws CarbonException { + TableInfo tableInfo = tableManager.getTable(identifier); + return CarbonTable.buildFromTableInfo(tableInfo); } @Override diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java similarity index 82% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java index 5afa381ba9b..dd7dca89052 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/DataService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service; +package org.apache.carbondata.store.impl.service; import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.sdk.store.service.model.BaseResponse; -import org.apache.carbondata.sdk.store.service.model.LoadDataRequest; -import org.apache.carbondata.sdk.store.service.model.ScanRequest; -import org.apache.carbondata.sdk.store.service.model.ScanResponse; +import org.apache.carbondata.store.impl.service.model.BaseResponse; +import org.apache.carbondata.store.impl.service.model.LoadDataRequest; +import org.apache.carbondata.store.impl.service.model.ScanRequest; +import org.apache.carbondata.store.impl.service.model.ScanResponse; import org.apache.hadoop.ipc.VersionedProtocol; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/PruneService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java similarity index 85% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/PruneService.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java index c1316da853e..997a79b1c81 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/PruneService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service; +package org.apache.carbondata.store.impl.service; import java.io.IOException; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.sdk.store.service.model.PruneRequest; -import org.apache.carbondata.sdk.store.service.model.PruneResponse; +import org.apache.carbondata.store.impl.service.model.PruneRequest; +import org.apache.carbondata.store.impl.service.model.PruneResponse; import org.apache.hadoop.ipc.VersionedProtocol; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/RegistryService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/RegistryService.java similarity index 84% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/RegistryService.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/RegistryService.java index a40f74174ed..f635050ee20 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/RegistryService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/RegistryService.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc; +package org.apache.carbondata.store.impl.service; import java.io.IOException; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerRequest; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerResponse; import org.apache.hadoop.ipc.VersionedProtocol; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/ServiceFactory.java similarity index 68% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/ServiceFactory.java index a9a14d5f1cc..41daf0f1a19 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/ServiceFactory.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/ServiceFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc; +package org.apache.carbondata.store.impl.service; import java.io.IOException; import java.net.InetAddress; @@ -35,4 +35,16 @@ public static RegistryService createRegistryService(String host, int port) throw RegistryService.class, RegistryService.versionID, address, new Configuration()); } + public static PruneService createPruneService(String host, int port) throws IOException { + InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); + return RPC.getProxy( + PruneService.class, PruneService.versionID, address, new Configuration()); + } + + public static DataService createDataService(String host, int port) throws IOException { + InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); + return RPC.getProxy( + DataService.class, DataService.versionID, address, new Configuration()); + } + } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/BaseResponse.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/BaseResponse.java similarity index 97% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/BaseResponse.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/BaseResponse.java index b1e87362f9f..49478e57622 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/BaseResponse.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/BaseResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/LoadDataRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/LoadDataRequest.java similarity index 97% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/LoadDataRequest.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/LoadDataRequest.java index 0956ef3e0c8..aa15fa940c9 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/LoadDataRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/LoadDataRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java similarity index 96% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneRequest.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java index 61d98260b9c..d3403a587d7 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneResponse.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneResponse.java similarity index 92% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneResponse.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneResponse.java index 7b071328ed2..111528b088f 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/PruneResponse.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; @@ -25,8 +25,8 @@ import java.util.List; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.sdk.store.BlockScanUnit; -import org.apache.carbondata.sdk.store.ScanUnit; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.impl.BlockScanUnit; import org.apache.hadoop.io.Writable; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/RegisterWorkerRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/RegisterWorkerRequest.java similarity index 97% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/RegisterWorkerRequest.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/RegisterWorkerRequest.java index 5f223d62052..54e727121b3 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/RegisterWorkerRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/RegisterWorkerRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/RegisterWorkerResponse.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/RegisterWorkerResponse.java similarity index 96% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/RegisterWorkerResponse.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/RegisterWorkerResponse.java index 16915e9c855..5bbfa67cf7e 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/RegisterWorkerResponse.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/RegisterWorkerResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ScanRequest.java similarity index 98% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ScanRequest.java index 209e3f186b8..d7b60ba0b16 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ScanRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ScanResponse.java similarity index 97% rename from store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ScanResponse.java index eaeafa50a72..c5a14c2d7af 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/model/ScanResponse.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ScanResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.sdk.store.service.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/ShutdownRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ShutdownRequest.java similarity index 96% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/ShutdownRequest.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ShutdownRequest.java index 311963d950a..d029d0ae441 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/ShutdownRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ShutdownRequest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/ShutdownResponse.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ShutdownResponse.java similarity index 96% rename from store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/ShutdownResponse.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ShutdownResponse.java index 0143f48cc3a..2650bbf88b3 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/rpc/model/ShutdownResponse.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/ShutdownResponse.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.carbondata.store.impl.rpc.model; +package org.apache.carbondata.store.impl.service.model; import java.io.DataInput; import java.io.DataOutput; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java index 23b9369bb09..3bed01d72f2 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java @@ -37,15 +37,15 @@ import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; import org.apache.carbondata.processing.loading.model.CarbonLoadModel; import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.service.model.BaseResponse; -import org.apache.carbondata.sdk.store.service.model.LoadDataRequest; -import org.apache.carbondata.sdk.store.service.model.ScanRequest; -import org.apache.carbondata.sdk.store.service.model.ScanResponse; import org.apache.carbondata.sdk.store.util.StoreUtil; import org.apache.carbondata.store.impl.Status; import org.apache.carbondata.store.impl.TableManager; -import org.apache.carbondata.store.impl.rpc.model.ShutdownRequest; -import org.apache.carbondata.store.impl.rpc.model.ShutdownResponse; +import org.apache.carbondata.store.impl.service.model.BaseResponse; +import org.apache.carbondata.store.impl.service.model.LoadDataRequest; +import org.apache.carbondata.store.impl.service.model.ScanRequest; +import org.apache.carbondata.store.impl.service.model.ScanResponse; +import org.apache.carbondata.store.impl.service.model.ShutdownRequest; +import org.apache.carbondata.store.impl.service.model.ShutdownResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java index fd50024eb4f..b47ee8ee60f 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java @@ -20,11 +20,11 @@ import java.io.IOException; import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.sdk.store.service.DataService; -import org.apache.carbondata.sdk.store.service.model.BaseResponse; -import org.apache.carbondata.sdk.store.service.model.LoadDataRequest; -import org.apache.carbondata.sdk.store.service.model.ScanRequest; -import org.apache.carbondata.sdk.store.service.model.ScanResponse; +import org.apache.carbondata.store.impl.service.DataService; +import org.apache.carbondata.store.impl.service.model.BaseResponse; +import org.apache.carbondata.store.impl.service.model.LoadDataRequest; +import org.apache.carbondata.store.impl.service.model.ScanRequest; +import org.apache.carbondata.store.impl.service.model.ScanResponse; import org.apache.hadoop.ipc.ProtocolSignature; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java index 5349456ed07..e3546d92f50 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/Worker.java @@ -24,12 +24,12 @@ import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.service.DataService; import org.apache.carbondata.sdk.store.util.StoreUtil; -import org.apache.carbondata.store.impl.rpc.RegistryService; -import org.apache.carbondata.store.impl.rpc.ServiceFactory; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerRequest; -import org.apache.carbondata.store.impl.rpc.model.RegisterWorkerResponse; +import org.apache.carbondata.store.impl.service.DataService; +import org.apache.carbondata.store.impl.service.RegistryService; +import org.apache.carbondata.store.impl.service.ServiceFactory; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerRequest; +import org.apache.carbondata.store.impl.service.model.RegisterWorkerResponse; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.RPC; diff --git a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java index 0677f1141ca..7f80a33859a 100644 --- a/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java +++ b/store/core/src/test/java/org/apache/carbondata/store/DistributedCarbonStoreTest.java @@ -73,7 +73,7 @@ public static void beforeAll() throws IOException, CarbonException, InterruptedE Thread.sleep(1000); - store = CarbonStoreFactory.getRemoteStore("DistributedCarbonStoreTest", storeConf); + store = CarbonStoreFactory.getDistributedStore("DistributedCarbonStoreTest", storeConf); } @AfterClass diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/HorizonClient.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/HorizonClient.java index eaa458331bb..e1b039e6e32 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/HorizonClient.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/HorizonClient.java @@ -28,7 +28,7 @@ import org.apache.carbondata.horizon.rest.model.view.DropTableRequest; import org.apache.carbondata.horizon.rest.model.view.LoadRequest; import org.apache.carbondata.horizon.rest.model.view.SelectRequest; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.exception.CarbonException; /** * Client to send REST request to Horizon service @@ -40,9 +40,9 @@ public interface HorizonClient extends Closeable { /** * Create a Table * @param create descriptor for create table operation - * @throws IOException if network or disk IO error occurs + * @throws CarbonException if network or disk IO error occurs */ - void createTable(CreateTableRequest create) throws IOException, StoreException; + void createTable(CreateTableRequest create) throws CarbonException; /** * Drop a Table, and remove all data in it @@ -56,7 +56,7 @@ public interface HorizonClient extends Closeable { * @param load descriptor for load operation * @throws IOException if network or disk IO error occurs */ - void loadData(LoadRequest load) throws IOException, StoreException; + void loadData(LoadRequest load) throws IOException, CarbonException; /** * Scan a Table and return matched rows @@ -64,7 +64,7 @@ public interface HorizonClient extends Closeable { * @return matched rows * @throws IOException if network or disk IO error occurs */ - List select(SelectRequest select) throws IOException, StoreException; + List select(SelectRequest select) throws IOException, CarbonException; /** * Executor a SQL statement diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/impl/SimpleHorizonClient.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/impl/SimpleHorizonClient.java index 076df70ffc8..b24c8d200f0 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/impl/SimpleHorizonClient.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/client/impl/SimpleHorizonClient.java @@ -31,7 +31,7 @@ import org.apache.carbondata.horizon.rest.model.view.LoadRequest; import org.apache.carbondata.horizon.rest.model.view.SelectRequest; import org.apache.carbondata.horizon.rest.model.view.SelectResponse; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.springframework.http.ResponseEntity; import org.springframework.web.client.RestTemplate; @@ -52,7 +52,7 @@ public SimpleHorizonClient(String serviceUri) { } @Override - public void createTable(CreateTableRequest create) throws IOException, StoreException { + public void createTable(CreateTableRequest create) throws CarbonException { Objects.requireNonNull(create); restTemplate.postForEntity(serviceUri + "/table/create", create, String.class); } @@ -64,18 +64,18 @@ public void dropTable(DropTableRequest drop) throws IOException { } @Override - public void loadData(LoadRequest load) throws IOException, StoreException { + public void loadData(LoadRequest load) throws IOException, CarbonException { Objects.requireNonNull(load); restTemplate.postForEntity(serviceUri + "/table/load", load, String.class); } @Override - public List select(SelectRequest select) throws IOException, StoreException { + public List select(SelectRequest select) throws IOException, CarbonException { Objects.requireNonNull(select); ResponseEntity response = restTemplate.postForEntity(serviceUri + "/table/select", select, SelectResponse.class); - Object[][] rows = Objects.requireNonNull(response.getBody()).getRows(); - List output = new ArrayList<>(rows.length); + List rows = Objects.requireNonNull(response.getBody()).getRows(); + List output = new ArrayList<>(rows.size()); for (Object[] row : rows) { output.add(new CarbonRow(row)); } diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/Horizon.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/Horizon.java index a30b5872e91..cffca07dc21 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/Horizon.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/Horizon.java @@ -17,7 +17,8 @@ package org.apache.carbondata.horizon.rest.controller; -import org.apache.carbondata.store.api.conf.StoreConf; + +import org.apache.carbondata.sdk.store.conf.StoreConf; import org.springframework.boot.SpringApplication; import org.springframework.boot.autoconfigure.SpringBootApplication; diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java index c625ec582f4..d3d1bd78668 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/controller/HorizonController.java @@ -18,7 +18,8 @@ import java.io.IOException; import java.net.InetAddress; -import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -36,14 +37,17 @@ import org.apache.carbondata.horizon.rest.model.view.LoadRequest; import org.apache.carbondata.horizon.rest.model.view.SelectRequest; import org.apache.carbondata.horizon.rest.model.view.SelectResponse; -import org.apache.carbondata.sdk.store.CarbonStore; -import org.apache.carbondata.sdk.store.CarbonStoreFactory; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.InternalCarbonStore; +import org.apache.carbondata.store.devapi.InternalCarbonStoreFactory; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.devapi.Scanner; import org.springframework.http.HttpStatus; import org.springframework.http.MediaType; @@ -59,7 +63,7 @@ public class HorizonController { private static LogService LOGGER = LogServiceFactory.getLogService(HorizonController.class.getName()); - private CarbonStore store; + private InternalCarbonStore store; public HorizonController() throws CarbonException { String storeFile = System.getProperty("carbonstore.conf.file"); @@ -67,7 +71,7 @@ public HorizonController() throws CarbonException { try { storeConf.conf(StoreConf.STORE_LOCATION, CarbonProperties.getStorePath()) .conf(StoreConf.MASTER_HOST, InetAddress.getLocalHost().getHostAddress()) - .conf(StoreConf.MASTER_PORT, CarbonProperties.getSearchMasterPort()) + .conf(StoreConf.STORE_PORT, CarbonProperties.getSearchMasterPort()) .conf(StoreConf.WORKER_HOST, InetAddress.getLocalHost().getHostAddress()) .conf(StoreConf.WORKER_PORT, CarbonProperties.getSearchWorkerPort()) .conf(StoreConf.WORKER_CORE_NUM, 2); @@ -76,13 +80,10 @@ public HorizonController() throws CarbonException { storeConf.load(storeFile); } - } catch (UnknownHostException e) { - throw new CarbonException(e); + store = InternalCarbonStoreFactory.getStore(storeConf); } catch (IOException e) { throw new CarbonException(e); } - - store = CarbonStoreFactory.getDistributedStore("GlobalStore", storeConf); } @RequestMapping(value = "echo") @@ -122,18 +123,25 @@ public ResponseEntity select(@RequestBody SelectRequest request) long start = System.currentTimeMillis(); RequestValidator.validateSelect(request); TableIdentifier table = new TableIdentifier(request.getTableName(), request.getDatabaseName()); - CarbonTable carbonTable = store.getTable(table); + CarbonTable carbonTable = store.getCarbonTable(table); Expression expression = Parser.parseFilter(request.getFilter(), carbonTable); - ScanDescriptor selectDescriptor = new ScanDescriptor( + Scanner scanner = store.newScanner(table); + List scanUnits = scanner.prune(table, expression); + ScanDescriptor scanDescriptor = new ScanDescriptor( table, request.getSelect(), expression, request.getLimit()); - List result = store.scan(selectDescriptor); - Iterator iterator = result.iterator(); - Object[][] output = new Object[result.size()][]; - int i = 0; - while (iterator.hasNext()) { - output[i] = (iterator.next().getData()); - i++; + ArrayList output = new ArrayList<>(); + for (ScanUnit scanUnit : scanUnits) { + Iterator> iterator = scanner.scan( + scanUnit, scanDescriptor, new HashMap()); + + while (iterator.hasNext()) { + ResultBatch rows = iterator.next(); + while (rows.hasNext()) { + output.add(rows.next().getData()); + } + } } + long end = System.currentTimeMillis(); LOGGER.audit("[" + request.getRequestId() + "] HorizonController select " + request.getDatabaseName() + "." + request.getTableName() + diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java index fbba57b7419..15a30cb3dd1 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java @@ -21,63 +21,63 @@ import org.apache.carbondata.horizon.rest.model.view.DropTableRequest; import org.apache.carbondata.horizon.rest.model.view.LoadRequest; import org.apache.carbondata.horizon.rest.model.view.SelectRequest; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.commons.lang.StringUtils; public class RequestValidator { - public static void validateSelect(SelectRequest request) throws StoreException { + public static void validateSelect(SelectRequest request) throws CarbonException { if (request == null) { - throw new StoreException("Select should not be null"); + throw new CarbonException("Select should not be null"); } if (StringUtils.isEmpty(request.getDatabaseName())) { - throw new StoreException("database name is invalid"); + throw new CarbonException("database name is invalid"); } if (StringUtils.isEmpty(request.getTableName())) { - throw new StoreException("table name is invalid"); + throw new CarbonException("table name is invalid"); } } - public static void validateTable(CreateTableRequest request) throws StoreException { + public static void validateTable(CreateTableRequest request) throws CarbonException { if (request == null) { - throw new StoreException("TableDescriptor should not be null"); + throw new CarbonException("TableDescriptor should not be null"); } if (StringUtils.isEmpty(request.getDatabaseName())) { - throw new StoreException("database name is invalid"); + throw new CarbonException("database name is invalid"); } if (StringUtils.isEmpty(request.getTableName())) { - throw new StoreException("table name is invalid"); + throw new CarbonException("table name is invalid"); } if (request.getFields() == null || request.getFields().length == 0) { - throw new StoreException("fields should not be empty"); + throw new CarbonException("fields should not be empty"); } } - public static void validateLoad(LoadRequest request) throws StoreException { + public static void validateLoad(LoadRequest request) throws CarbonException { if (request == null) { - throw new StoreException("LoadDescriptor should not be null"); + throw new CarbonException("LoadDescriptor should not be null"); } if (StringUtils.isEmpty(request.getDatabaseName())) { - throw new StoreException("database name is invalid"); + throw new CarbonException("database name is invalid"); } if (StringUtils.isEmpty(request.getTableName())) { - throw new StoreException("table name is invalid"); + throw new CarbonException("table name is invalid"); } if (StringUtils.isEmpty(request.getInputPath())) { - throw new StoreException("input path is invalid"); + throw new CarbonException("input path is invalid"); } } - public static void validateDrop(DropTableRequest request) throws StoreException { + public static void validateDrop(DropTableRequest request) throws CarbonException { if (request == null) { - throw new StoreException("DropTableRequest should not be null"); + throw new CarbonException("DropTableRequest should not be null"); } if (StringUtils.isEmpty(request.getDatabaseName())) { - throw new StoreException("database name is invalid"); + throw new CarbonException("database name is invalid"); } if (StringUtils.isEmpty(request.getTableName())) { - throw new StoreException("table name is invalid"); + throw new CarbonException("table name is invalid"); } } } diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/FieldRequest.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/FieldRequest.java index b809d9ea168..200b3a29e5e 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/FieldRequest.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/FieldRequest.java @@ -108,7 +108,7 @@ public Field convertToDto() { field.setPrecision(precision); field.setScale(scale); field.setColumnComment(comment); - field.setChildren(new LinkedList()); + field.setChildren(new LinkedList()); return field; } } diff --git a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/SelectResponse.java b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/SelectResponse.java index 6bf5c75ea32..e4f8cf8ac21 100644 --- a/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/SelectResponse.java +++ b/store/horizon/src/main/java/org/apache/carbondata/horizon/rest/model/view/SelectResponse.java @@ -17,23 +17,22 @@ package org.apache.carbondata.horizon.rest.model.view; +import java.util.List; + public class SelectResponse extends Response { - private Object[][] rows; + private List rows; public SelectResponse() { } - public SelectResponse(SelectRequest request, String message, Object[][] rows) { + public SelectResponse(SelectRequest request, String message, List rows) { super(request, message); this.rows = rows; } - public Object[][] getRows() { + public List getRows() { return rows; } - public void setRows(Object[][] rows) { - this.rows = rows; - } } diff --git a/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java b/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java index b980cb064f7..59c4bd12a1c 100644 --- a/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java +++ b/store/horizon/src/test/java/org/apache/carbondata/horizon/HorizonTest.java @@ -32,7 +32,7 @@ import org.apache.carbondata.horizon.rest.model.view.SelectRequest; import org.apache.carbondata.horizon.rest.model.view.SelectResponse; import org.apache.carbondata.store.api.conf.StoreConf; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.store.api.exception.CarbonException; import org.apache.carbondata.store.impl.worker.Worker; import org.apache.carbondata.sdk.store.util.StoreUtil; @@ -109,13 +109,13 @@ public void testHorizon() { SelectRequest select = createSelectRequest(5, null, "intField", "stringField"); SelectResponse result = restTemplate.postForObject(serviceUri + "/table/select", select, SelectResponse.class); - Assert.assertEquals(5, result.getRows().length); + Assert.assertEquals(5, result.getRows().size()); // select row with filter SelectRequest filter = createSelectRequest(5, "intField = 11", "intField", "stringField"); SelectResponse filterResult = restTemplate.postForObject(serviceUri + "/table/select", filter, SelectResponse.class); - Assert.assertEquals(1, filterResult.getRows().length); + Assert.assertEquals(1, filterResult.getRows().size()); request = createDropTableRequest(); response = restTemplate.postForObject(serviceUri + "/table/drop", request, String.class); @@ -173,7 +173,7 @@ private CreateTableRequest createCreateTableRequest() { } @Test - public void testHorizonClient() throws IOException, StoreException { + public void testHorizonClient() throws IOException, CarbonException { HorizonClient client = new SimpleHorizonClient(serviceUri); DropTableRequest drop = createDropTableRequest(); client.dropTable(drop); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java index 9f390c80b6b..b8a6f4478a9 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -18,11 +18,13 @@ package org.apache.carbondata.sdk.store; import java.io.Closeable; +import java.util.Iterator; import java.util.List; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.datatype.StructType; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; @@ -88,14 +90,6 @@ public interface CarbonStore extends Closeable { */ void loadData(LoadDescriptor load) throws CarbonException; - /** - * Return a new Loader that can be used to load data in distributed compute framework - * @param load descriptor for load operation - * @return a new Loader - * @throws CarbonException if any error occurs - */ - Loader newLoader(LoadDescriptor load) throws CarbonException; - /** * Return true if this table has primary key defined when create table using * {@link #createTable(TableDescriptor)} @@ -110,14 +104,19 @@ default boolean isPrimaryKeyDefined() { } /** - * Return a new mutator that supports realtime insert, upsert and delete using primary key - * - * @return a new mutator + * Insert a batch of rows if key is not exist, otherwise update the row + * @param row rows to be upsert + * @param schema schema of the input row (fields without the primary key) * @throws CarbonException if any error occurs */ - default Mutator newMutator(TableIdentifier tableIdentifier) throws CarbonException { - throw new UnsupportedOperationException(); - } + void upsert(Iterator row, StructType schema) throws CarbonException; + + /** + * Delete a batch of rows + * @param keys keys to be deleted + * @throws CarbonException if any error occurs + */ + void delete(Iterator keys) throws CarbonException; //////////////////////////////////////////////////////////////////// @@ -134,21 +133,20 @@ default Mutator newMutator(TableIdentifier tableIdentifier) throws CarbonExcepti List scan(ScanDescriptor select) throws CarbonException; /** - * Return a new Scanner that can be used in for parallel scan - * - * @param tableIdentifier table to scan - * @return a new Scanner + * Lookup and return a row with specified primary key + * @param key key to lookup + * @return matched row for the specified key * @throws CarbonException if any error occurs */ - Scanner newScanner(TableIdentifier tableIdentifier) throws CarbonException; + Row lookup(PrimaryKey key) throws CarbonException; /** - * Return a new Fetch that can be used for lookup operation + * Lookup by filter expression and return a list of matched row * - * @return a new Fetcher + * @param tableIdentifier table identifier + * @param filterExpression filter expression, like "col3 = 1" + * @return matched row for the specified filter * @throws CarbonException if any error occurs */ - default Fetcher newFetcher(TableIdentifier tableIdentifier) throws CarbonException { - throw new UnsupportedOperationException(); - } + List lookup(TableIdentifier tableIdentifier, String filterExpression) throws CarbonException; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java index b09b8af7907..6f5bbb59723 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java @@ -40,7 +40,7 @@ public class CarbonStoreFactory { private CarbonStoreFactory() { } - public static CarbonStore getRemoteStore(String storeName, StoreConf storeConf) + public static CarbonStore getDistributedStore(String storeName, StoreConf storeConf) throws CarbonException { if (remoteStore.containsKey(storeName)) { return remoteStore.get(storeName); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java index 806348cf94b..a7f0d6191dc 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java @@ -18,14 +18,15 @@ package org.apache.carbondata.sdk.store; import java.io.IOException; +import java.util.Iterator; import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; +import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.metadata.datatype.StructType; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; @@ -38,21 +39,17 @@ /** * A CarbonStore that leverage multiple servers via RPC calls (Master and Workers) */ +@InterfaceAudience.User public class DistributedCarbonStore implements CarbonStore { private static final LogService LOGGER = LogServiceFactory.getLogService(DistributedCarbonStore.class.getCanonicalName()); - private static final long versionID = 1L; - - private StoreService storeService; - private StoreConf storeConf; - private Map tableCache = new ConcurrentHashMap<>(); + protected StoreService storeService; public DistributedCarbonStore(StoreConf conf) throws IOException { this.storeService = ServiceFactory.createStoreService(conf.masterHost(), conf.storeServicePort()); - this.storeConf = conf; } @Override @@ -80,8 +77,8 @@ public List listTable() throws CarbonException { @Override public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { - TableInfo tableInfo = storeService.getTable(table); - // TODO: create TableDescriptor from tableInfo + CarbonTable carbonTable = storeService.getTable(table); + // TODO: create TableDescriptor from carbonTable return null; } @@ -96,7 +93,12 @@ public void loadData(LoadDescriptor load) throws CarbonException { } @Override - public Loader newLoader(LoadDescriptor load) throws CarbonException { + public void upsert(Iterator row, StructType schema) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public void delete(Iterator keys) throws CarbonException { throw new UnsupportedOperationException(); } @@ -111,14 +113,14 @@ public List scan(ScanDescriptor select) throws CarbonException { } @Override - public Scanner newScanner(TableIdentifier identifier) throws CarbonException { - TableInfo tableInfo = tableCache.getOrDefault(identifier, storeService.getTable(identifier)); - tableCache.putIfAbsent(identifier, tableInfo); - try { - return new RowScanner(storeConf, tableInfo); - } catch (IOException e) { - throw new CarbonException(e); - } + public Row lookup(PrimaryKey key) throws CarbonException { + throw new UnsupportedOperationException(); + } + + @Override + public List lookup(TableIdentifier tableIdentifier, String filterExpression) + throws CarbonException { + throw new UnsupportedOperationException(); } @Override diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java deleted file mode 100644 index cc26e9a4058..00000000000 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Fetcher.java +++ /dev/null @@ -1,37 +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.carbondata.sdk.store; - -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.sdk.store.exception.CarbonException; - -/** - * A Fetcher is used to lookup row by primary key - */ -@InterfaceAudience.User -@InterfaceStability.Unstable -public interface Fetcher { - /** - * Lookup and return a row with specified primary key - * @param key key to lookup - * @return matched row for the specified key - * @throws CarbonException if any error occurs - */ - Row lookup(PrimaryKey key) throws CarbonException; -} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java deleted file mode 100644 index 28dc9a2e70a..00000000000 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/Mutator.java +++ /dev/null @@ -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.carbondata.sdk.store; - -import java.util.Iterator; - -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.core.metadata.datatype.StructType; -import org.apache.carbondata.sdk.store.exception.CarbonException; - -/** - * A Mutator is used to perform insert, update, delete operation on the table - */ -@InterfaceAudience.User -@InterfaceStability.Unstable -public interface Mutator extends TransactionalOperation { - - /** - * Insert a batch of rows if key is not exist, otherwise update the row - * @param row rows to be upsert - * @param schema schema of the input row (fields without the primary key) - * @throws CarbonException if any error occurs - */ - void upsert(Iterator row, StructType schema) throws CarbonException; - - /** - * Delete a batch of rows - * @param keys keys to be deleted - * @throws CarbonException if any error occurs - */ - void delete(Iterator keys) throws CarbonException; -} diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java index e3721d5c571..925ed91b912 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/ServiceFactory.java @@ -34,23 +34,8 @@ public static StoreService createStoreService(String host, int port) throws IOEx return RPC.getProxy(StoreService.class, StoreService.versionID, address, new Configuration()); } - public static PruneService createPruneService(String host, int port) throws IOException { - InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); - return RPC.getProxy( - PruneService.class, PruneService.versionID, address, new Configuration()); - } - - public static DataService createDataService(String host, int port) throws IOException { - InetSocketAddress address = new InetSocketAddress(InetAddress.getByName(host), port); - return RPC.getProxy( - DataService.class, DataService.versionID, address, new Configuration()); - } - public static void stopStoreService(StoreService service) { RPC.stopProxy(service); } - public static void stopDataService(DataService service) { - RPC.stopProxy(service); - } } \ No newline at end of file diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java index 7e54d2f5548..fa5f0656c1f 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java @@ -21,7 +21,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; @@ -38,7 +38,7 @@ public interface StoreService extends VersionedProtocol { void dropTable(TableIdentifier table) throws CarbonException; - TableInfo getTable(TableIdentifier table) throws CarbonException; + CarbonTable getTable(TableIdentifier table) throws CarbonException; List listTable() throws CarbonException; diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java index d0128fc263e..d654590eec8 100644 --- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java +++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java @@ -29,16 +29,14 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.util.path.CarbonTablePath; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.CharEncoding; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.apache.avro.Schema; - public class AvroCarbonWriterTest { private String path = "./AvroCarbonWriterSuiteWriteFiles"; diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java index 19bf194c680..75e4da9391d 100644 --- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java +++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java @@ -17,12 +17,15 @@ package org.apache.carbondata.sdk.file; -import java.io.*; +import java.io.File; +import java.io.FileFilter; +import java.io.FilenameFilter; +import java.io.IOException; import java.sql.Date; import java.sql.Timestamp; -import java.util.*; +import java.util.Arrays; +import java.util.Comparator; -import org.apache.avro.generic.GenericData; import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; @@ -30,16 +33,25 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.LiteralExpression; -import org.apache.carbondata.core.scan.expression.conditional.*; +import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; +import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression; +import org.apache.carbondata.core.scan.expression.conditional.InExpression; +import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression; +import org.apache.carbondata.core.scan.expression.conditional.NotEqualsExpression; +import org.apache.carbondata.core.scan.expression.conditional.NotInExpression; import org.apache.carbondata.core.scan.expression.logical.AndExpression; import org.apache.carbondata.core.scan.expression.logical.OrExpression; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.path.CarbonTablePath; import junit.framework.TestCase; +import org.apache.avro.generic.GenericData; import org.apache.commons.io.FileUtils; -import org.apache.commons.lang.CharEncoding; -import org.junit.*; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; public class CarbonReaderTest extends TestCase { diff --git a/store/sql/src/main/java/org/apache/carbondata/horizon/rest/controller/SqlHorizonController.java b/store/sql/src/main/java/org/apache/carbondata/horizon/rest/controller/SqlHorizonController.java index 4c3b99622c2..1910e041e33 100644 --- a/store/sql/src/main/java/org/apache/carbondata/horizon/rest/controller/SqlHorizonController.java +++ b/store/sql/src/main/java/org/apache/carbondata/horizon/rest/controller/SqlHorizonController.java @@ -26,7 +26,7 @@ import org.apache.carbondata.horizon.rest.model.view.SqlRequest; import org.apache.carbondata.horizon.rest.model.view.SqlResponse; import org.apache.carbondata.horizon.rest.sql.SparkSqlWrapper; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.store.api.exception.CarbonException; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Dataset; @@ -46,7 +46,7 @@ public class SqlHorizonController { LogServiceFactory.getLogService(SqlHorizonController.class.getName()); @RequestMapping(value = "/table/sql", produces = MediaType.APPLICATION_JSON_VALUE) - public ResponseEntity sql(@RequestBody SqlRequest request) throws StoreException { + public ResponseEntity sql(@RequestBody SqlRequest request) throws CarbonException { RequestValidator.validateSql(request); List rows; Dataset sqlDataFrame = null; @@ -56,10 +56,10 @@ public ResponseEntity sql(@RequestBody SqlRequest request) throws S rows = sqlDataFrame.collectAsList(); } catch (AnalysisException e) { LOGGER.error(e); - throw new StoreException(e.getSimpleMessage()); + throw new CarbonException(e.getSimpleMessage()); } catch (Exception e) { LOGGER.error(e); - throw new StoreException(e.getMessage()); + throw new CarbonException(e.getMessage()); } final String[] fieldNames = sqlDataFrame.schema().fieldNames(); Object[][] responseData = new Object[0][]; diff --git a/store/sql/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java b/store/sql/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java index 82e095aa1bf..dc82d6d329b 100644 --- a/store/sql/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java +++ b/store/sql/src/main/java/org/apache/carbondata/horizon/rest/model/validate/RequestValidator.java @@ -18,18 +18,18 @@ package org.apache.carbondata.horizon.rest.model.validate; import org.apache.carbondata.horizon.rest.model.view.SqlRequest; -import org.apache.carbondata.store.api.exception.StoreException; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.commons.lang.StringUtils; public class RequestValidator { - public static void validateSql(SqlRequest request) throws StoreException { + public static void validateSql(SqlRequest request) throws CarbonException { if (request == null) { - throw new StoreException("Select should not be null"); + throw new CarbonException("Select should not be null"); } if (StringUtils.isEmpty(request.getSqlStatement())) { - throw new StoreException("sql statement is invalid"); + throw new CarbonException("sql statement is invalid"); } } } From cea22739a06ce21f7f67f5a1d9bf5a68664e2913 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sat, 4 Aug 2018 19:40:53 +0800 Subject: [PATCH 12/14] support remote prune and local scan --- .../carbondata/hadoop/CarbonInputSplit.java | 1 - .../hadoop/readsupport/CarbonReadSupport.java | 3 +- .../devapi/{Loader.java => DataLoader.java} | 2 +- .../carbondata/store/devapi/DataScanner.java | 44 +++++ .../store/devapi/InternalCarbonStore.java | 13 +- .../devapi/InternalCarbonStoreFactory.java | 15 +- .../store/devapi/InternalCarbonStoreImpl.java | 63 ------- .../carbondata/store/devapi/Pruner.java | 42 +++++ .../carbondata/store/devapi/ScanOption.java | 43 ++++- .../carbondata/store/devapi/Scanner.java | 50 +----- .../carbondata/store/impl/DataOperation.java | 95 ++++++++++ .../carbondata/store/impl/IndexOperation.java | 61 +++++++ .../store/impl/InternalCarbonStoreImpl.java | 122 +++++++++++++ .../store/impl/LocalCarbonStore.java | 18 +- .../store/impl/LocalDataScanner.java | 69 ++++++++ .../carbondata/store/impl/LocalPruner.java | 58 ++++++ .../{TableManager.java => MetaOperation.java} | 121 ++----------- .../store/impl/RemoteDataScanner.java | 83 +++++++++ .../carbondata/store/impl/RemotePruner.java | 56 ++++++ .../{RowScanner.java => RemoteScanner.java} | 28 ++- .../carbondata/store/impl/master/Master.java | 25 ++- .../store/impl/master/PruneServiceImpl.java | 56 +++--- .../store/impl/master/StoreServiceImpl.java | 22 ++- .../store/impl/service/DataService.java | 4 +- .../store/impl/service/PruneService.java | 5 +- .../impl/service/model/PruneRequest.java | 29 ++- .../store/impl/worker/DataRequestHandler.java | 166 ------------------ .../store/impl/worker/DataServiceImpl.java | 121 ++++++++++++- .../carbondata/sdk/store/CarbonStore.java | 5 +- .../sdk/store/CarbonStoreFactory.java | 24 +-- .../sdk/store/DistributedCarbonStore.java | 6 +- .../sdk/store/descriptor/ScanDescriptor.java | 3 +- .../sdk/store/service/StoreService.java | 4 +- .../sdk/file/AvroCarbonWriterTest.java | 4 +- .../carbondata/sdk/file/CarbonReaderTest.java | 24 +-- 35 files changed, 964 insertions(+), 521 deletions(-) rename store/core/src/main/java/org/apache/carbondata/store/devapi/{Loader.java => DataLoader.java} (95%) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/DataOperation.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/IndexOperation.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/LocalPruner.java rename store/core/src/main/java/org/apache/carbondata/store/impl/{TableManager.java => MetaOperation.java} (64%) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/RemotePruner.java rename store/core/src/main/java/org/apache/carbondata/store/impl/{RowScanner.java => RemoteScanner.java} (83%) delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java index 7f1c8b11443..dd3e63f6ec6 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/CarbonInputSplit.java @@ -280,7 +280,6 @@ public Segment getSegment() { out.writeLong(getStart()); out.writeLong(getLength()); } - super.write(out); out.writeUTF(segment.toString()); out.writeShort(version.number()); out.writeUTF(bucketId); diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java index c126e9539ce..78a2e7fa9fb 100644 --- a/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java +++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/readsupport/CarbonReadSupport.java @@ -17,6 +17,7 @@ package org.apache.carbondata.hadoop.readsupport; import java.io.IOException; +import java.io.Serializable; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn; @@ -24,7 +25,7 @@ /** * This is the interface to convert data reading from RecordReader to row representation. */ -public interface CarbonReadSupport { +public interface CarbonReadSupport extends Serializable { /** * Initialization if needed based on the projected column list diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/Loader.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java similarity index 95% rename from store/core/src/main/java/org/apache/carbondata/store/devapi/Loader.java rename to store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java index d34fa2d7c93..7e8aefac863 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/Loader.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java @@ -31,7 +31,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface Loader extends TransactionalOperation, Serializable { +public interface DataLoader extends TransactionalOperation, Serializable { /** * Trigger the load operation * @throws CarbonException if any error occurs diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java new file mode 100644 index 00000000000..58a90dc55eb --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java @@ -0,0 +1,44 @@ +/* + * 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.carbondata.store.devapi; + +import java.util.Iterator; + +import org.apache.carbondata.sdk.store.exception.CarbonException; + +public interface DataScanner { + + /** + * Perform a scan in a distributed compute framework like Spark, Presto, etc. + * Filter/Projection/Limit operation is pushed down to the scan. + * + * This should be used with {@link Pruner#prune(TableIdentifier, Expression)} + * in a distributed compute environment. It enables the framework to + * do a parallel scan by creating multiple {@link ScanUnit} and perform + * parallel scan in worker, such as Spark executor + * + * The return result is in batch so that the caller can start next + * level of computation before getting all results, such as + * implementing a `prefetch` execution model. + * + * @param input one scan unit + * @return scan result, the result is returned in batch + * @throws CarbonException if any error occurs + */ + Iterator> scan(ScanUnit input) throws CarbonException; +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java index 1c1b5d94203..01c20086286 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStore.java @@ -17,12 +17,15 @@ package org.apache.carbondata.store.devapi; +import java.util.Map; + import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; import org.apache.carbondata.sdk.store.CarbonStore; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; @@ -48,18 +51,22 @@ public interface InternalCarbonStore extends CarbonStore { * @return a new Loader * @throws CarbonException if any error occurs */ - Loader newLoader(LoadDescriptor load) throws CarbonException; + DataLoader newLoader(LoadDescriptor load) throws CarbonException; /** * Return a new Scanner that can be used in for parallel scan * * @param tableIdentifier table to scan + * @param scanOption options for scan, use {@link ScanOption} for the map key * @param readSupport read support to convert the row to output object * @param the target object type contain in {@link ResultBatch} * @return a new Scanner * @throws CarbonException if any error occurs */ - Scanner newRowScanner(TableIdentifier tableIdentifier, CarbonReadSupport readSupport) - throws CarbonException; + Scanner newScanner( + TableIdentifier tableIdentifier, + ScanDescriptor scanDescriptor, + Map scanOption, + CarbonReadSupport readSupport) throws CarbonException; } diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java index f729064e7df..6a4644426b7 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java @@ -18,13 +18,24 @@ package org.apache.carbondata.store.devapi; import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.store.impl.InternalCarbonStoreImpl; public class InternalCarbonStoreFactory { - public static InternalCarbonStore getStore(StoreConf conf) throws IOException { - return new InternalCarbonStoreImpl(conf); + private static final Map stores = new ConcurrentHashMap<>(); + + public static synchronized InternalCarbonStore getStore(String storeName, StoreConf conf) + throws IOException { + InternalCarbonStore store = stores.getOrDefault(storeName, newStore(conf)); + stores.putIfAbsent(storeName, store); + return store; } + private static InternalCarbonStore newStore(StoreConf conf) throws IOException { + return new InternalCarbonStoreImpl(conf); + } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java deleted file mode 100644 index 0d172179ce9..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreImpl.java +++ /dev/null @@ -1,63 +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.carbondata.store.devapi; - -import java.io.IOException; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; -import org.apache.carbondata.sdk.store.DistributedCarbonStore; -import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; -import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; -import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.store.impl.RowScanner; - -public class InternalCarbonStoreImpl extends DistributedCarbonStore implements InternalCarbonStore { - - private Map tableCache = new ConcurrentHashMap<>(); - private StoreConf storeConf; - - InternalCarbonStoreImpl(StoreConf storeConf) throws IOException { - super(storeConf); - this.storeConf = storeConf; - } - - @Override - public CarbonTable getCarbonTable(TableIdentifier tableIdentifier) - throws CarbonException { - CarbonTable carbonTable = tableCache.getOrDefault( - tableIdentifier, storeService.getTable(tableIdentifier)); - tableCache.putIfAbsent(tableIdentifier, carbonTable); - return carbonTable; - } - - @Override - public Loader newLoader(LoadDescriptor load) throws CarbonException { - throw new UnsupportedOperationException(); - } - - @Override - public Scanner newRowScanner(TableIdentifier identifier, CarbonReadSupport readSupport) - throws CarbonException { - return new RowScanner<>(storeConf, getCarbonTable(identifier), readSupport); - } - -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java new file mode 100644 index 00000000000..cc15865a7e0 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java @@ -0,0 +1,42 @@ +/* + * 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.carbondata.store.devapi; + +import java.util.List; + +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; + +public interface Pruner { + + /** + * Return an array of ScanUnit which will be the input in + * {@link Scanner#scan(ScanUnit)} + * + * Implementation will leverage index to prune using specified + * filter expression + * + * @param table table identifier + * @param filterExpression expression of filter predicate given by user + * @return list of ScanUnit which should be passed to + * {@link Scanner#scan(ScanUnit)} + * @throws CarbonException if any error occurs + */ + List prune(TableIdentifier table, Expression filterExpression) throws CarbonException; +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java index fdd4ec78d2d..795a2db5b2a 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java @@ -17,12 +17,53 @@ package org.apache.carbondata.store.devapi; +import java.util.Map; + import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; @InterfaceAudience.User @InterfaceStability.Unstable public class ScanOption { + + /** batch size in number of rows in one {@link ResultBatch} */ public static final String BATCH_SIZE = "batchSize"; - public static final String ENABLE_ROW_MAJOR = "rowMajor"; + + /** + * set to true if return in row major object in {@link ResultBatch}, + * otherwise columnar object is returned + */ + public static final String ROW_MAJOR = "rowMajor"; + + /** + * set to true if enable remote prune by RPC call, + * otherwise prune executes in caller's JVM + */ + public static final String REMOTE_PRUNE = "remotePrune"; + + /** + * set to true if enable operator pushdown like scan and load + * otherwise operation executes in caller's JVM + */ + public static final String OP_PUSHDOWN = "operatorPushDown"; + + /** + * Return true if REMOTE_PRUNE is set, default is false + */ + public static boolean isRemotePrune(Map options) { + if (options == null) { + return false; + } + return Boolean.valueOf(options.getOrDefault(REMOTE_PRUNE, "false")); + } + + /** + * Return true if REMOTE_PRUNE is set, default is false + */ + public static boolean isOperatorPushdown(Map options) { + if (options == null) { + return false; + } + return Boolean.valueOf(options.getOrDefault(OP_PUSHDOWN, "false")); + } } \ No newline at end of file diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java index ab1c9e2dc64..217be1346d6 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java @@ -18,16 +18,9 @@ package org.apache.carbondata.store.devapi; import java.io.Serializable; -import java.util.Iterator; -import java.util.List; -import java.util.Map; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; -import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; -import org.apache.carbondata.sdk.store.exception.CarbonException; /** * A Scanner is used to scan the table in a distributed compute @@ -35,47 +28,6 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public interface Scanner extends Serializable { - - /** - * Return an array of ScanUnit which will be the input in - * {@link #scan(ScanUnit, ScanDescriptor, Map)} - * - * Implementation will leverage index to prune using specified - * filter expression - * - * @param table table identifier - * @param filterExpression expression of filter predicate given by user - * @return list of ScanUnit which should be passed to - * {@link #scan(ScanUnit, ScanDescriptor, Map)} - * @throws CarbonException if any error occurs - */ - List prune( - TableIdentifier table, - Expression filterExpression) throws CarbonException; - - /** - * Perform a scan in a distributed compute framework like Spark, Presto, etc. - * Filter/Projection/Limit operation is pushed down to the scan. - * - * This should be used with {@link #prune(TableIdentifier, Expression)} - * in a distributed compute environment. It enables the framework to - * do a parallel scan by creating multiple {@link ScanUnit} and perform - * parallel scan in worker, such as Spark executor - * - * The return result is in batch so that the caller can start next - * level of computation before getting all results, such as - * implementing a `prefetch` execution model. - * - * @param input one scan unit - * @param scanDescriptor parameter for scanning - * @param scanOption options for scan, use {@link ScanOption} for the map key - * @return scan result, the result is returned in batch - * @throws CarbonException if any error occurs - */ - Iterator> scan( - ScanUnit input, - ScanDescriptor scanDescriptor, - Map scanOption) throws CarbonException; +public interface Scanner extends DataScanner, Pruner, Serializable { } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DataOperation.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DataOperation.java new file mode 100644 index 00000000000..fc50a62fe2a --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DataOperation.java @@ -0,0 +1,95 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; + +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.core.scan.model.QueryModel; +import org.apache.carbondata.core.scan.model.QueryModelBuilder; +import org.apache.carbondata.core.util.CarbonTaskInfo; +import org.apache.carbondata.core.util.ThreadLocalTaskInfo; +import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; +import org.apache.carbondata.hadoop.CarbonRecordReader; +import org.apache.carbondata.store.impl.service.model.ScanRequest; + +public class DataOperation { + + private static final LogService LOGGER = + LogServiceFactory.getLogService(DataOperation.class.getCanonicalName()); + + /** + * Scan data and return matched rows. This should be invoked in worker side. + * @param tableInfo carbon table + * @param scan scan parameter + * @return matched rows + * @throws IOException if IO error occurs + */ + public static List scan(TableInfo tableInfo, ScanRequest scan) throws IOException { + CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo); + CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo(); + carbonTaskInfo.setTaskId(System.nanoTime()); + ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo); + + CarbonMultiBlockSplit mbSplit = scan.getSplit(); + long limit = scan.getLimit(); + QueryModel queryModel = createQueryModel(table, scan); + + LOGGER.info(String.format("[QueryId:%d] %s, number of block: %d", scan.getRequestId(), + queryModel.toString(), mbSplit.getAllSplits().size())); + + // read all rows by the reader + List rows = new LinkedList<>(); + try (CarbonRecordReader reader = new IndexedRecordReader(scan.getRequestId(), + table, queryModel)) { + reader.initialize(mbSplit, null); + + // loop to read required number of rows. + // By default, if user does not specify the limit value, limit is Long.MaxValue + long rowCount = 0; + while (reader.nextKeyValue() && rowCount < limit) { + rows.add(reader.getCurrentValue()); + rowCount++; + } + } catch (InterruptedException e) { + throw new IOException(e); + } + LOGGER.info(String.format("[QueryId:%d] scan completed, return %d rows", + scan.getRequestId(), rows.size())); + return rows; + } + + private static QueryModel createQueryModel(CarbonTable table, ScanRequest scan) { + String[] projectColumns = scan.getProjectColumns(); + Expression filter = null; + if (scan.getFilterExpression() != null) { + filter = scan.getFilterExpression(); + } + return new QueryModelBuilder(table) + .projectColumns(projectColumns) + .filterExpression(filter) + .build(); + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/IndexOperation.java b/store/core/src/main/java/org/apache/carbondata/store/impl/IndexOperation.java new file mode 100644 index 00000000000..49afb454f15 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/IndexOperation.java @@ -0,0 +1,61 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +import org.apache.carbondata.core.exception.InvalidConfigurationException; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.hadoop.api.CarbonInputFormat; +import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; +import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.Job; + +public class IndexOperation { + + /** + * Prune data by leveraging Carbon's Index + */ + public static List pruneBlock(TableInfo tableInfo, Expression filter) + throws IOException { + Objects.requireNonNull(tableInfo); + CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo); + JobConf jobConf = new JobConf(new Configuration()); + Job job = new Job(jobConf); + CarbonTableInputFormat format; + try { + // We just want to do pruning, so passing empty projection columns + format = CarbonInputFormatUtil.createCarbonTableInputFormat( + job, table, new String[0], filter, null, null, true); + } catch (InvalidConfigurationException e) { + throw new IOException(e.getMessage()); + } + + // We will do FG pruning in reader side, so don't do it here + CarbonInputFormat.setFgDataMapPruning(job.getConfiguration(), false); + return format.getSplits(job); + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java new file mode 100644 index 00000000000..bb3f3fc10e4 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java @@ -0,0 +1,122 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; +import org.apache.carbondata.sdk.store.DistributedCarbonStore; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.DataLoader; +import org.apache.carbondata.store.devapi.DataScanner; +import org.apache.carbondata.store.devapi.InternalCarbonStore; +import org.apache.carbondata.store.devapi.Pruner; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanOption; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.devapi.Scanner; + +/** + * This store does prune and scan either remotely by sending RPC to Master/Worker + * or in local JVM, depends on parameter passed. + */ +@InterfaceAudience.Internal +public class InternalCarbonStoreImpl extends DistributedCarbonStore implements InternalCarbonStore { + + private Map tableCache = new ConcurrentHashMap<>(); + private StoreConf storeConf; + + public InternalCarbonStoreImpl(StoreConf storeConf) throws IOException { + super(storeConf); + this.storeConf = storeConf; + } + + @Override + public CarbonTable getCarbonTable(TableIdentifier tableIdentifier) + throws CarbonException { + Objects.requireNonNull(tableIdentifier); + CarbonTable carbonTable = tableCache.getOrDefault( + tableIdentifier, + CarbonTable.buildFromTableInfo(storeService.getTable(tableIdentifier))); + tableCache.putIfAbsent(tableIdentifier, carbonTable); + return carbonTable; + } + + @Override + public DataLoader newLoader(LoadDescriptor load) throws CarbonException { + throw new UnsupportedOperationException(); + } + + /** + * By default, it returns a Scanner that does prune and scan remotely + */ + @Override + public Scanner newScanner(TableIdentifier identifier, ScanDescriptor scanDescriptor, + Map scanOption, CarbonReadSupport readSupport) + throws CarbonException { + Objects.requireNonNull(identifier); + Objects.requireNonNull(scanDescriptor); + if (scanOption == null) { + return new RemoteScanner<>(storeConf, getCarbonTable(identifier), scanDescriptor, scanOption, + readSupport); + } + TableInfo tableInfo = MetaOperation.getTable(identifier, storeConf); + boolean isRemotePrune = ScanOption.isRemotePrune(scanOption); + boolean isOpPushdown = ScanOption.isOperatorPushdown(scanOption); + + Pruner pruner; + DataScanner scanner; + + if (isRemotePrune) { + pruner = new RemotePruner(storeConf.masterHost(), storeConf.pruneServicePort()); + } else { + pruner = new LocalPruner(storeConf); + } + if (isOpPushdown) { + scanner = new RemoteDataScanner<>(tableInfo, scanDescriptor, scanOption, readSupport); + } else { + scanner = new LocalDataScanner<>(storeConf, scanDescriptor, scanOption); + } + + return new Scanner() { + @Override public List prune(TableIdentifier table, Expression filterExpression) + throws CarbonException { + return pruner.prune(table, filterExpression); + } + + @Override public Iterator> scan(ScanUnit input) + throws CarbonException { + return scanner.scan(input); + } + }; + } + +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java index b55e70b6a85..bd8fafe43a1 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalCarbonStore.java @@ -29,13 +29,13 @@ import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.block.Distributable; import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.metadata.datatype.StructType; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.core.mutate.CarbonUpdateUtil; import org.apache.carbondata.core.util.ThreadLocalTaskInfo; +import org.apache.carbondata.hadoop.CarbonInputSplit; import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; import org.apache.carbondata.processing.loading.DataLoadExecutor; @@ -71,7 +71,7 @@ */ @InterfaceAudience.User @InterfaceStability.Unstable -public class LocalCarbonStore extends TableManager implements CarbonStore { +public class LocalCarbonStore extends MetaOperation implements CarbonStore { private static final LogService LOGGER = LogServiceFactory.getLogService(LocalCarbonStore.class.getName()); @@ -84,7 +84,7 @@ public LocalCarbonStore(StoreConf storeConf) { this(storeConf, new Configuration()); } - public LocalCarbonStore(StoreConf storeConf, Configuration hadoopConf) { + private LocalCarbonStore(StoreConf storeConf, Configuration hadoopConf) { super(storeConf); this.storeConf = storeConf; this.txnManager = SegmentTxnManager.getInstance(); @@ -96,7 +96,7 @@ public void loadData(LoadDescriptor load) throws CarbonException { Objects.requireNonNull(load); CarbonLoadModel loadModel; try { - TableInfo tableInfo = getTable(load.getTable()); + TableInfo tableInfo = getTable(load.getTable(), storeConf); CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo); CarbonLoadModelBuilder modelBuilder = new CarbonLoadModelBuilder(table); modelBuilder.setInputPath(load.getInputPath()); @@ -175,14 +175,14 @@ private void loadData(CarbonLoadModel model) throws Exception { public List scan(ScanDescriptor scanDescriptor) throws CarbonException { Objects.requireNonNull(scanDescriptor); try { - TableInfo tableInfo = getTable(scanDescriptor.getTableIdentifier()); - CarbonTable table = CarbonTable.buildFromTableInfo(tableInfo); - List blocks = pruneBlock(table, scanDescriptor.getFilter()); - CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, ""); + TableInfo tableInfo = getTable(scanDescriptor.getTableIdentifier(), storeConf); + List blocks = + IndexOperation.pruneBlock(tableInfo, scanDescriptor.getFilter()); + CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, new String[0]); ScanRequest scan = new ScanRequest(0, split, tableInfo, scanDescriptor.getProjection(), scanDescriptor.getFilter(), scanDescriptor.getLimit()); - return scan(table, scan); + return DataOperation.scan(tableInfo, scan); } catch (IOException e) { throw new CarbonException(e); } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java new file mode 100644 index 00000000000..ed329c22c30 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java @@ -0,0 +1,69 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.DataScanner; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.impl.service.model.ScanRequest; + +public class LocalDataScanner implements DataScanner { + + private StoreConf storeConf; + private ScanDescriptor scanDescriptor; + private Map scanOption; + + LocalDataScanner(StoreConf storeConf, ScanDescriptor scanDescriptor, + Map scanOption) { + this.storeConf = storeConf; + this.scanDescriptor = scanDescriptor; + this.scanOption = scanOption; + } + + @Override + public Iterator> scan(ScanUnit input) throws CarbonException { + Objects.requireNonNull(scanDescriptor); + try { + TableInfo tableInfo = MetaOperation.getTable(scanDescriptor.getTableIdentifier(), storeConf); + List blocks = + IndexOperation.pruneBlock(tableInfo, scanDescriptor.getFilter()); + CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(blocks, new String[0]); + ScanRequest scan = + new ScanRequest(0, split, tableInfo, scanDescriptor.getProjection(), + scanDescriptor.getFilter(), scanDescriptor.getLimit()); + List rows = (List) DataOperation.scan(tableInfo, scan); + RowMajorResultBatch resultBatch = new RowMajorResultBatch<>(rows); + return Collections.singletonList(resultBatch).iterator(); + } catch (IOException e) { + throw new CarbonException(e); + } + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalPruner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalPruner.java new file mode 100644 index 00000000000..734072c8643 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalPruner.java @@ -0,0 +1,58 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.Pruner; +import org.apache.carbondata.store.devapi.ScanUnit; + +public class LocalPruner implements Pruner { + + private StoreConf storeConf; + + LocalPruner(StoreConf storeConf) { + this.storeConf = storeConf; + } + + @Override + public List prune(TableIdentifier identifier, Expression filterExpression) + throws CarbonException { + try { + TableInfo table = MetaOperation.getTable(identifier, storeConf); + List splits = IndexOperation.pruneBlock(table, filterExpression); + return splits.stream().map( + (Function) inputSplit -> + // LocalScanner will scan in local JVM, it is not sending RPC to + // schedulable (Worker), so it can be null + new BlockScanUnit(inputSplit, null) + ).collect(Collectors.toList()); + } catch (IOException e) { + throw new CarbonException(e); + } + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java b/store/core/src/main/java/org/apache/carbondata/store/impl/MetaOperation.java similarity index 64% rename from store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/MetaOperation.java index 92a79caff38..3ca1ffe7379 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/TableManager.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/MetaOperation.java @@ -18,10 +18,8 @@ package org.apache.carbondata.store.impl; import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Objects; @@ -30,10 +28,7 @@ import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.block.Distributable; import org.apache.carbondata.core.datastore.impl.FileFactory; -import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.exception.InvalidConfigurationException; import org.apache.carbondata.core.fileoperations.FileWriteOperation; import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier; import org.apache.carbondata.core.metadata.converter.SchemaConverter; @@ -44,47 +39,31 @@ import org.apache.carbondata.core.metadata.schema.table.TableSchema; import org.apache.carbondata.core.metadata.schema.table.TableSchemaBuilder; import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema; -import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.core.scan.model.QueryModel; -import org.apache.carbondata.core.scan.model.QueryModelBuilder; -import org.apache.carbondata.core.util.CarbonTaskInfo; import org.apache.carbondata.core.util.CarbonUtil; -import org.apache.carbondata.core.util.ThreadLocalTaskInfo; import org.apache.carbondata.core.util.path.CarbonTablePath; import org.apache.carbondata.core.writer.ThriftWriter; -import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; -import org.apache.carbondata.hadoop.CarbonRecordReader; -import org.apache.carbondata.hadoop.api.CarbonInputFormat; -import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; -import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; import org.apache.carbondata.sdk.file.CarbonWriterBuilder; import org.apache.carbondata.sdk.file.Field; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.store.impl.service.model.ScanRequest; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; /** * Provides table management. */ @InterfaceAudience.Internal -public class TableManager { +public class MetaOperation { - private static LogService LOGGER = - LogServiceFactory.getLogService(TableManager.class.getCanonicalName()); + private static final LogService LOGGER = + LogServiceFactory.getLogService(MetaOperation.class.getCanonicalName()); private StoreConf storeConf; // mapping of table path to CarbonTable object - private Map cache = new HashMap<>(); + private static final Map cache = new HashMap<>(); - public TableManager(StoreConf storeConf) { + public MetaOperation(StoreConf storeConf) { this.storeConf = storeConf; } @@ -184,7 +163,12 @@ public void dropTable(TableIdentifier table) throws CarbonException { } public TableInfo getTable(TableIdentifier table) throws CarbonException { - String tablePath = getTablePath(table.getTableName(), table.getDatabaseName()); + return getTable(table, storeConf); + } + + public static TableInfo getTable(TableIdentifier table, StoreConf storeConf) + throws CarbonException { + String tablePath = getTablePath(table.getTableName(), table.getDatabaseName(), storeConf); if (cache.containsKey(tablePath)) { return cache.get(tablePath); } else { @@ -220,84 +204,9 @@ public String getTablePath(String tableName, String databaseName) { Objects.requireNonNull(databaseName); return String.format("%s/%s", storeConf.storeLocation(), tableName); } - - /** - * Prune data by using CarbonInputFormat.getSplit - * Return a mapping of host address to list of block. - * This should be invoked in driver side. - */ - static List pruneBlock(CarbonTable table, Expression filter) throws IOException { - Objects.requireNonNull(table); - JobConf jobConf = new JobConf(new Configuration()); - Job job = new Job(jobConf); - CarbonTableInputFormat format; - try { - // We just want to do pruning, so passing empty projection columns - format = CarbonInputFormatUtil.createCarbonTableInputFormat( - job, table, new String[0], filter, null, null, true); - } catch (InvalidConfigurationException e) { - throw new IOException(e.getMessage()); - } - - // We will do FG pruning in reader side, so don't do it here - CarbonInputFormat.setFgDataMapPruning(job.getConfiguration(), false); - List splits = format.getSplits(job); - List blockInfos = new ArrayList<>(splits.size()); - for (InputSplit split : splits) { - blockInfos.add((Distributable) split); - } - return blockInfos; - } - - /** - * Scan data and return matched rows. This should be invoked in worker side. - * @param table carbon table - * @param scan scan parameter - * @return matched rows - * @throws IOException if IO error occurs - */ - public static List scan(CarbonTable table, ScanRequest scan) throws IOException { - CarbonTaskInfo carbonTaskInfo = new CarbonTaskInfo(); - carbonTaskInfo.setTaskId(System.nanoTime()); - ThreadLocalTaskInfo.setCarbonTaskInfo(carbonTaskInfo); - - CarbonMultiBlockSplit mbSplit = scan.getSplit(); - long limit = scan.getLimit(); - QueryModel queryModel = createQueryModel(table, scan); - - LOGGER.info(String.format("[QueryId:%d] %s, number of block: %d", scan.getRequestId(), - queryModel.toString(), mbSplit.getAllSplits().size())); - - // read all rows by the reader - List rows = new LinkedList<>(); - try (CarbonRecordReader reader = new IndexedRecordReader(scan.getRequestId(), - table, queryModel)) { - reader.initialize(mbSplit, null); - - // loop to read required number of rows. - // By default, if user does not specify the limit value, limit is Long.MaxValue - long rowCount = 0; - while (reader.nextKeyValue() && rowCount < limit) { - rows.add(reader.getCurrentValue()); - rowCount++; - } - } catch (InterruptedException e) { - throw new IOException(e); - } - LOGGER.info(String.format("[QueryId:%d] scan completed, return %d rows", - scan.getRequestId(), rows.size())); - return rows; - } - - private static QueryModel createQueryModel(CarbonTable table, ScanRequest scan) { - String[] projectColumns = scan.getProjectColumns(); - Expression filter = null; - if (scan.getFilterExpression() != null) { - filter = scan.getFilterExpression(); - } - return new QueryModelBuilder(table) - .projectColumns(projectColumns) - .filterExpression(filter) - .build(); + public static String getTablePath(String tableName, String databaseName, StoreConf storeConf) { + Objects.requireNonNull(tableName); + Objects.requireNonNull(databaseName); + return String.format("%s/%s", storeConf.storeLocation(), tableName); } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java new file mode 100644 index 00000000000..8c5edf22641 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java @@ -0,0 +1,83 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.stream.Collectors; + +import org.apache.carbondata.core.metadata.schema.table.TableInfo; +import org.apache.carbondata.hadoop.CarbonInputSplit; +import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; +import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; +import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.DataScanner; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.impl.service.DataService; +import org.apache.carbondata.store.impl.service.model.ScanRequest; +import org.apache.carbondata.store.impl.service.model.ScanResponse; + +public class RemoteDataScanner implements DataScanner { + + private TableInfo tableInfo; + private ScanDescriptor scanDescriptor; + private Map scanOption; + private CarbonReadSupport readSupport; + + RemoteDataScanner(TableInfo tableInfo, ScanDescriptor scanDescriptor, + Map scanOption, CarbonReadSupport readSupport) { + this.tableInfo = tableInfo; + this.scanDescriptor = scanDescriptor; + this.scanOption = scanOption; + this.readSupport = readSupport; + } + + @Override + public Iterator> scan(ScanUnit input) throws CarbonException { + List toBeScan = new ArrayList<>(); + if (input instanceof BlockScanUnit) { + toBeScan.add(((BlockScanUnit) input).getInputSplit()); + } else { + throw new CarbonException(input.getClass().getName() + " is not supported"); + } + int queryId = new Random().nextInt(); + CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(toBeScan, input.preferredLocations()); + try { + ScanRequest request = new ScanRequest(queryId, split, tableInfo, + scanDescriptor.getProjection(), scanDescriptor.getFilter(), scanDescriptor.getLimit()); + DataService dataService = + DataServicePool.getOrCreateDataService(((BlockScanUnit) input).getSchedulable()); + ScanResponse response = dataService.scan(request); + List rows = Arrays.stream(response.getRows()) + .map(row -> readSupport.readRow(row)) + .collect(Collectors.toList()); + + return Collections.singletonList(new RowMajorResultBatch(rows)).iterator(); + } catch (IOException e) { + throw new CarbonException(e); + } + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/RemotePruner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RemotePruner.java new file mode 100644 index 00000000000..5fc485e3982 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RemotePruner.java @@ -0,0 +1,56 @@ +/* + * 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.carbondata.store.impl; + +import java.io.IOException; +import java.util.List; + +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.Pruner; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.impl.service.PruneService; +import org.apache.carbondata.store.impl.service.ServiceFactory; +import org.apache.carbondata.store.impl.service.model.PruneRequest; +import org.apache.carbondata.store.impl.service.model.PruneResponse; + +public class RemotePruner implements Pruner { + + private String pruneServiceHost; + private int pruneServiePort; + + RemotePruner(String pruneServiceHost, int pruneServiePort) { + this.pruneServiceHost = pruneServiceHost; + this.pruneServiePort = pruneServiePort; + } + + @Override + public List prune(TableIdentifier table, Expression filterExpression) + throws CarbonException { + try { + PruneRequest request = new PruneRequest(table, filterExpression); + PruneService pruneService = ServiceFactory.createPruneService( + pruneServiceHost, pruneServiePort); + PruneResponse response = pruneService.prune(request); + return response.getScanUnits(); + } catch (IOException e) { + throw new CarbonException(e); + } + } +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/RowScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java similarity index 83% rename from store/core/src/main/java/org/apache/carbondata/store/impl/RowScanner.java rename to store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java index 692ba9f7919..d8035add353 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/RowScanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java @@ -34,7 +34,6 @@ import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.hadoop.CarbonInputSplit; import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; -import org.apache.carbondata.hadoop.api.CarbonInputFormat; import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; @@ -51,21 +50,27 @@ import org.apache.carbondata.store.impl.service.model.ScanRequest; import org.apache.carbondata.store.impl.service.model.ScanResponse; -import org.apache.hadoop.conf.Configuration; - -public class RowScanner implements Scanner { +/** + * This Scanner executes pruning and scanning in remote Master and Worker + */ +public class RemoteScanner implements Scanner { private static final LogService LOGGER = - LogServiceFactory.getLogService(RowScanner.class.getCanonicalName()); + LogServiceFactory.getLogService(RemoteScanner.class.getCanonicalName()); private TableInfo tableInfo; private String pruneServiceHost; private int pruneServiePort; + private ScanDescriptor scanDescriptor; + private Map scanOption; private CarbonReadSupport readSupport; - public RowScanner(StoreConf conf, CarbonTable carbonTable, CarbonReadSupport readSupport) { + RemoteScanner(StoreConf conf, CarbonTable carbonTable, ScanDescriptor scanDescriptor, + Map scanOption, CarbonReadSupport readSupport) { this.tableInfo = carbonTable.getTableInfo(); this.pruneServiceHost = conf.masterHost(); this.pruneServiePort = conf.pruneServicePort(); + this.scanDescriptor = scanDescriptor; + this.scanOption = scanOption; this.readSupport = readSupport; } @@ -73,11 +78,7 @@ public RowScanner(StoreConf conf, CarbonTable carbonTable, CarbonReadSupport public List prune(TableIdentifier table, Expression filterExpression) throws CarbonException { try { - Configuration configuration = new Configuration(); - CarbonInputFormat.setTableName(configuration, table.getTableName()); - CarbonInputFormat.setDatabaseName(configuration, table.getDatabaseName()); - CarbonInputFormat.setFilterPredicates(configuration, filterExpression); - PruneRequest request = new PruneRequest(configuration); + PruneRequest request = new PruneRequest(table, filterExpression); PruneService pruneService = ServiceFactory.createPruneService( pruneServiceHost, pruneServiePort); PruneResponse response = pruneService.prune(request); @@ -88,10 +89,7 @@ public List prune(TableIdentifier table, Expression filterExpression) } @Override - public Iterator> scan( - ScanUnit input, - ScanDescriptor scanDescriptor, - Map option) throws CarbonException { + public Iterator> scan(ScanUnit input) throws CarbonException { List toBeScan = new ArrayList<>(); if (input instanceof BlockScanUnit) { toBeScan.add(((BlockScanUnit) input).getInputSplit()); diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java index 214c89aab1b..dbb0116bf63 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/Master.java @@ -51,7 +51,7 @@ public class Master { private static LogService LOGGER = LogServiceFactory.getLogService(Master.class.getName()); - private StoreConf conf; + private StoreConf storeConf; private Configuration hadoopConf; private RPC.Server registryServer = null; private RPC.Server pruneServer = null; @@ -60,9 +60,9 @@ public class Master { // mapping of worker IP address to worker instance Map workers = new ConcurrentHashMap<>(); - public Master(StoreConf conf) { - this.conf = conf; - this.hadoopConf = conf.newHadoopConf(); + public Master(StoreConf storeConf) { + this.storeConf = storeConf; + this.hadoopConf = storeConf.newHadoopConf(); } /** @@ -73,8 +73,8 @@ private void startRegistryService() throws IOException { BindException exception; // we will try to create service at worse case 100 times int numTry = 100; - String host = conf.masterHost(); - int port = conf.registryServicePort(); + String host = storeConf.masterHost(); + int port = storeConf.registryServicePort(); LOGGER.info("building registry-service on " + host + ":" + port); RegistryService registryService = new RegistryServiceImpl(this); @@ -124,11 +124,11 @@ private void startPruneService() throws IOException { BindException exception; // we will try to create service at worse case 100 times int numTry = 100; - String host = conf.masterHost(); - int port = conf.pruneServicePort(); + String host = storeConf.masterHost(); + int port = storeConf.pruneServicePort(); LOGGER.info("building prune-service on " + host + ":" + port); - PruneService pruneService = new PruneServiceImpl(); + PruneService pruneService = new PruneServiceImpl(storeConf, new Scheduler(this)); do { try { pruneServer = new RPC.Builder(hadoopConf) @@ -137,7 +137,6 @@ private void startPruneService() throws IOException { .setProtocol(PruneService.class) .setInstance(pruneService) .build(); - ((PruneServiceImpl) pruneService).setScheduler(new Scheduler(this)); pruneServer.start(); numTry = 0; exception = null; @@ -175,11 +174,11 @@ private void startStoreService() throws IOException { BindException exception; // we will try to create service at worse case 100 times int numTry = 100; - String host = conf.masterHost(); - int port = conf.storeServicePort(); + String host = storeConf.masterHost(); + int port = storeConf.storeServicePort(); LOGGER.info("building store-service on " + host + ":" + port); - StoreService storeService = new StoreServiceImpl(conf); + StoreService storeService = new StoreServiceImpl(storeConf); do { try { storeServer = new RPC.Builder(hadoopConf) diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java index f9f8af95acf..726c4421352 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/PruneServiceImpl.java @@ -22,46 +22,58 @@ import java.util.function.Function; import java.util.stream.Collectors; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.hadoop.CarbonInputSplit; -import org.apache.carbondata.hadoop.api.CarbonTableInputFormat; +import org.apache.carbondata.sdk.store.conf.StoreConf; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.store.devapi.ScanUnit; import org.apache.carbondata.store.impl.BlockScanUnit; +import org.apache.carbondata.store.impl.IndexOperation; +import org.apache.carbondata.store.impl.MetaOperation; import org.apache.carbondata.store.impl.Schedulable; import org.apache.carbondata.store.impl.service.PruneService; import org.apache.carbondata.store.impl.service.model.PruneRequest; import org.apache.carbondata.store.impl.service.model.PruneResponse; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtocolSignature; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; public class PruneServiceImpl implements PruneService { + private StoreConf storeConf; private Scheduler scheduler; - public void setScheduler(Scheduler scheduler) { + PruneServiceImpl(StoreConf storeConf, Scheduler scheduler) { + this.storeConf = storeConf; this.scheduler = scheduler; } @Override - public PruneResponse prune(PruneRequest request) throws IOException { - Configuration hadoopConf = request.getHadoopConf(); - Job job = new Job(hadoopConf); - CarbonTableInputFormat format = new CarbonTableInputFormat(hadoopConf); - List prunedResult = format.getSplits(job); - List output = - prunedResult.stream().map((Function) inputSplit -> { - String[] locations = ((CarbonInputSplit) inputSplit).preferredLocations(); - Schedulable worker; - if (locations.length == 0) { - worker = scheduler.pickNexWorker(); - } else { - worker = scheduler.pickWorker(locations[0]); - } - return new BlockScanUnit((CarbonInputSplit) inputSplit, worker); - }).collect(Collectors.toList()); - return new PruneResponse(output); + public PruneResponse prune(PruneRequest request) throws CarbonException { + TableInfo tableInfo = MetaOperation.getTable(request.getTable(), storeConf); + try { + List splits = + IndexOperation.pruneBlock(tableInfo, request.getFilterExpression()); + List output = splits.stream() + .map((Function) inputSplit -> { + Schedulable worker; + String[] locations; + try { + locations = inputSplit.getLocations(); + if (locations.length == 0) { + worker = scheduler.pickNexWorker(); + } else { + worker = scheduler.pickWorker(locations[0]); + } + } catch (IOException e) { + // ignore it and pick next worker as no locality + worker = scheduler.pickNexWorker(); + } + return new BlockScanUnit(inputSplit, worker); + }).collect(Collectors.toList()); + return new PruneResponse(output); + } catch (IOException e) { + throw new CarbonException(e); + } } @Override diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java index 06798265636..5d0fd889291 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/StoreServiceImpl.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; @@ -31,7 +30,7 @@ import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.sdk.store.service.StoreService; import org.apache.carbondata.store.impl.LocalCarbonStore; -import org.apache.carbondata.store.impl.TableManager; +import org.apache.carbondata.store.impl.MetaOperation; import org.apache.hadoop.ipc.ProtocolSignature; @@ -40,43 +39,42 @@ public class StoreServiceImpl implements StoreService { // TODO: simple implementation, load and scan inside master private LocalCarbonStore localStore; - private TableManager tableManager; + private MetaOperation metaOperation; StoreServiceImpl(StoreConf storeConf) { localStore = new LocalCarbonStore(storeConf); - tableManager = new TableManager(storeConf); + metaOperation = new MetaOperation(storeConf); } @Override public void createTable(TableDescriptor descriptor) throws CarbonException { - tableManager.createTable(descriptor); + metaOperation.createTable(descriptor); } @Override public void dropTable(TableIdentifier identifier) throws CarbonException { - tableManager.dropTable(identifier); + metaOperation.dropTable(identifier); } @Override - public CarbonTable getTable(TableIdentifier identifier) throws CarbonException { - TableInfo tableInfo = tableManager.getTable(identifier); - return CarbonTable.buildFromTableInfo(tableInfo); + public TableInfo getTable(TableIdentifier identifier) throws CarbonException { + return metaOperation.getTable(identifier); } @Override public List listTable() throws CarbonException { - return tableManager.listTable(); + return metaOperation.listTable(); } @Override public TableDescriptor getDescriptor(TableIdentifier identifier) throws CarbonException { - return tableManager.getDescriptor(identifier); + return metaOperation.getDescriptor(identifier); } @Override public void alterTable(TableIdentifier identifier, TableDescriptor newTable) throws CarbonException { - tableManager.alterTable(identifier, newTable); + metaOperation.alterTable(identifier, newTable); } @Override diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java index dd7dca89052..de28252a48c 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java @@ -17,6 +17,8 @@ package org.apache.carbondata.store.impl.service; +import java.io.Closeable; + import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.store.impl.service.model.BaseResponse; import org.apache.carbondata.store.impl.service.model.LoadDataRequest; @@ -25,7 +27,7 @@ import org.apache.hadoop.ipc.VersionedProtocol; -public interface DataService extends VersionedProtocol { +public interface DataService extends VersionedProtocol, Closeable { long versionID = 1L; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java index 997a79b1c81..1f6306e2ac4 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java @@ -17,9 +17,8 @@ package org.apache.carbondata.store.impl.service; -import java.io.IOException; - import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.sdk.store.exception.CarbonException; import org.apache.carbondata.store.impl.service.model.PruneRequest; import org.apache.carbondata.store.impl.service.model.PruneResponse; @@ -28,5 +27,5 @@ @InterfaceAudience.Internal public interface PruneService extends VersionedProtocol { long versionID = 1L; - PruneResponse prune(PruneRequest request) throws IOException; + PruneResponse prune(PruneRequest request) throws CarbonException; } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java index d3403a587d7..ace35f32c4a 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/model/PruneRequest.java @@ -22,32 +22,43 @@ import java.io.IOException; import java.io.Serializable; -import org.apache.hadoop.conf.Configuration; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.core.util.ObjectSerializationUtil; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; + import org.apache.hadoop.io.Writable; public class PruneRequest implements Serializable, Writable { - private Configuration hadoopConf; + private TableIdentifier table; + private Expression filterExpression; public PruneRequest() { } - public PruneRequest(Configuration hadoopConf) { - this.hadoopConf = hadoopConf; + public PruneRequest(TableIdentifier table, Expression filterExpression) { + this.table = table; + this.filterExpression = filterExpression; + } + + public TableIdentifier getTable() { + return table; } - public Configuration getHadoopConf() { - return hadoopConf; + public Expression getFilterExpression() { + return filterExpression; } @Override public void write(DataOutput out) throws IOException { - hadoopConf.write(out); + table.write(out); + out.writeUTF(ObjectSerializationUtil.convertObjectToString(filterExpression)); } @Override public void readFields(DataInput in) throws IOException { - this.hadoopConf = new Configuration(); - this.hadoopConf.readFields(in); + table = new TableIdentifier(); + table.readFields(in); + filterExpression = (Expression) (ObjectSerializationUtil.convertStringToObject(in.readUTF())); } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java deleted file mode 100644 index 3bed01d72f2..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataRequestHandler.java +++ /dev/null @@ -1,166 +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.carbondata.store.impl.worker; - -import java.io.IOException; -import java.util.Date; -import java.util.Iterator; -import java.util.List; - -import org.apache.carbondata.common.CarbonIterator; -import org.apache.carbondata.common.annotations.InterfaceAudience; -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.scan.executor.impl.SearchModeDetailQueryExecutor; -import org.apache.carbondata.core.scan.executor.impl.SearchModeVectorDetailQueryExecutor; -import org.apache.carbondata.core.util.ThreadLocalTaskInfo; -import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; -import org.apache.carbondata.processing.loading.DataLoadExecutor; -import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat; -import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; -import org.apache.carbondata.processing.loading.model.CarbonLoadModel; -import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.util.StoreUtil; -import org.apache.carbondata.store.impl.Status; -import org.apache.carbondata.store.impl.TableManager; -import org.apache.carbondata.store.impl.service.model.BaseResponse; -import org.apache.carbondata.store.impl.service.model.LoadDataRequest; -import org.apache.carbondata.store.impl.service.model.ScanRequest; -import org.apache.carbondata.store.impl.service.model.ScanResponse; -import org.apache.carbondata.store.impl.service.model.ShutdownRequest; -import org.apache.carbondata.store.impl.service.model.ShutdownResponse; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.JobID; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.mapreduce.TaskID; -import org.apache.hadoop.mapreduce.TaskType; -import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; - -/** - * It handles data request from client. - */ -@InterfaceAudience.Internal -class DataRequestHandler { - - private StoreConf storeConf; - private Configuration hadoopConf; - - DataRequestHandler(StoreConf conf, Configuration hadoopConf) { - this.storeConf = conf; - this.hadoopConf = hadoopConf; - } - - private static final LogService LOGGER = - LogServiceFactory.getLogService(DataRequestHandler.class.getName()); - - ScanResponse handleScan(ScanRequest scan) { - try { - LOGGER.info(String.format("[QueryId:%d] receive search request", scan.getRequestId())); - CarbonTable table = CarbonTable.buildFromTableInfo(scan.getTableInfo()); - List rows = TableManager.scan(table, scan); - LOGGER.info(String.format("[QueryId:%d] sending success response", scan.getRequestId())); - return createSuccessResponse(scan, rows); - } catch (IOException e) { - LOGGER.error(e); - LOGGER.info(String.format("[QueryId:%d] sending failure response", scan.getRequestId())); - return createFailureResponse(scan, e); - } - } - - ShutdownResponse handleShutdown(ShutdownRequest request) { - LOGGER.info("Shutting down worker..."); - SearchModeDetailQueryExecutor.shutdownThreadPool(); - SearchModeVectorDetailQueryExecutor.shutdownThreadPool(); - LOGGER.info("Worker shut down"); - return new ShutdownResponse(Status.SUCCESS.ordinal(), ""); - } - - /** - * create a failure response - */ - private ScanResponse createFailureResponse(ScanRequest scan, Throwable throwable) { - return new ScanResponse(scan.getRequestId(), Status.FAILURE.ordinal(), - throwable.getMessage(), new Object[0][]); - } - - /** - * create a success response with result rows - */ - private ScanResponse createSuccessResponse(ScanRequest scan, List rows) { - Iterator itor = rows.iterator(); - Object[][] output = new Object[rows.size()][]; - int i = 0; - while (itor.hasNext()) { - output[i++] = itor.next().getData(); - } - return new ScanResponse(scan.getRequestId(), Status.SUCCESS.ordinal(), "", output); - } - - public BaseResponse handleLoadData(LoadDataRequest request) { - DataLoadExecutor executor = null; - try { - CarbonLoadModel model = request.getModel(); - - JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0); - CarbonInputFormatUtil.createJobTrackerID(new Date()); - TaskID taskId = new TaskID(jobId, TaskType.MAP, 0); - TaskAttemptID taskAttemptId = new TaskAttemptID(taskId, 0); - Configuration configuration = new Configuration(hadoopConf); - StoreUtil.configureCSVInputFormat(configuration, model); - configuration.set(FileInputFormat.INPUT_DIR, model.getFactFilePath()); - // Set up the attempt context required to use in the output committer. - TaskAttemptContext hadoopAttemptContext = - new TaskAttemptContextImpl(configuration, taskAttemptId); - - CSVInputFormat format = new CSVInputFormat(); - List splits = format.getSplits(hadoopAttemptContext); - - CarbonIterator[] readerIterators = new CSVRecordReaderIterator[splits.size()]; - for (int index = 0; index < splits.size(); index++) { - readerIterators[index] = new CSVRecordReaderIterator( - format.createRecordReader(splits.get(index), hadoopAttemptContext), splits.get(index), - hadoopAttemptContext); - } - - executor = new DataLoadExecutor(); - executor.execute(model, storeConf.storeTempLocation(), readerIterators); - - return new BaseResponse(Status.SUCCESS.ordinal(), ""); - } catch (IOException e) { - LOGGER.error(e, "Failed to handle load data"); - return new BaseResponse(Status.FAILURE.ordinal(), e.getMessage()); - } catch (InterruptedException e) { - LOGGER.error(e, "Interrupted handle load data "); - return new BaseResponse(Status.FAILURE.ordinal(), e.getMessage()); - } catch (Exception e) { - LOGGER.error(e, "Failed to execute load data "); - return new BaseResponse(Status.FAILURE.ordinal(), e.getMessage()); - } finally { - if (executor != null) { - executor.close(); - StoreUtil.clearUnsafeMemory(ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId()); - } - } - } -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java index b47ee8ee60f..d1b8f43ebeb 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/worker/DataServiceImpl.java @@ -18,33 +18,139 @@ package org.apache.carbondata.store.impl.worker; import java.io.IOException; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import org.apache.carbondata.common.CarbonIterator; import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.datastore.row.CarbonRow; +import org.apache.carbondata.core.scan.executor.impl.SearchModeDetailQueryExecutor; +import org.apache.carbondata.core.scan.executor.impl.SearchModeVectorDetailQueryExecutor; +import org.apache.carbondata.core.util.ThreadLocalTaskInfo; +import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil; +import org.apache.carbondata.processing.loading.DataLoadExecutor; +import org.apache.carbondata.processing.loading.csvinput.CSVInputFormat; +import org.apache.carbondata.processing.loading.csvinput.CSVRecordReaderIterator; +import org.apache.carbondata.processing.loading.model.CarbonLoadModel; +import org.apache.carbondata.sdk.store.util.StoreUtil; +import org.apache.carbondata.store.impl.DataOperation; +import org.apache.carbondata.store.impl.Status; import org.apache.carbondata.store.impl.service.DataService; import org.apache.carbondata.store.impl.service.model.BaseResponse; import org.apache.carbondata.store.impl.service.model.LoadDataRequest; import org.apache.carbondata.store.impl.service.model.ScanRequest; import org.apache.carbondata.store.impl.service.model.ScanResponse; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.ipc.ProtocolSignature; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.mapreduce.TaskID; +import org.apache.hadoop.mapreduce.TaskType; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; @InterfaceAudience.Internal public class DataServiceImpl implements DataService { - private DataRequestHandler handler; + private static final LogService LOGGER = + LogServiceFactory.getLogService(DataServiceImpl.class.getCanonicalName()); + + // temp location for loading (writing sort temp files) + private String[] storeTempLocation; + private Configuration hadoopConf; DataServiceImpl(Worker worker) { - this.handler = new DataRequestHandler(worker.getConf(), worker.getHadoopConf()); + this.hadoopConf = worker.getHadoopConf(); + this.storeTempLocation = worker.getConf().storeTempLocation(); } @Override public BaseResponse loadData(LoadDataRequest request) { - return handler.handleLoadData(request); + DataLoadExecutor executor = null; + try { + CarbonLoadModel model = request.getModel(); + + JobID jobId = CarbonInputFormatUtil.getJobId(new Date(), 0); + CarbonInputFormatUtil.createJobTrackerID(new Date()); + TaskID taskId = new TaskID(jobId, TaskType.MAP, 0); + TaskAttemptID taskAttemptId = new TaskAttemptID(taskId, 0); + Configuration configuration = new Configuration(hadoopConf); + StoreUtil.configureCSVInputFormat(configuration, model); + configuration.set(FileInputFormat.INPUT_DIR, model.getFactFilePath()); + // Set up the attempt context required to use in the output committer. + TaskAttemptContext hadoopAttemptContext = + new TaskAttemptContextImpl(configuration, taskAttemptId); + + CSVInputFormat format = new CSVInputFormat(); + List splits = format.getSplits(hadoopAttemptContext); + + CarbonIterator[] readerIterators = new CSVRecordReaderIterator[splits.size()]; + for (int index = 0; index < splits.size(); index++) { + readerIterators[index] = new CSVRecordReaderIterator( + format.createRecordReader(splits.get(index), hadoopAttemptContext), splits.get(index), + hadoopAttemptContext); + } + + executor = new DataLoadExecutor(); + executor.execute(model, storeTempLocation, readerIterators); + + return new BaseResponse(Status.SUCCESS.ordinal(), ""); + } catch (IOException e) { + LOGGER.error(e, "Failed to handle load data"); + return new BaseResponse(Status.FAILURE.ordinal(), e.getMessage()); + } catch (InterruptedException e) { + LOGGER.error(e, "Interrupted handle load data "); + return new BaseResponse(Status.FAILURE.ordinal(), e.getMessage()); + } catch (Exception e) { + LOGGER.error(e, "Failed to execute load data "); + return new BaseResponse(Status.FAILURE.ordinal(), e.getMessage()); + } finally { + if (executor != null) { + executor.close(); + StoreUtil.clearUnsafeMemory(ThreadLocalTaskInfo.getCarbonTaskInfo().getTaskId()); + } + } } @Override public ScanResponse scan(ScanRequest scan) { - return handler.handleScan(scan); + try { + LOGGER.info(String.format("[QueryId:%d] receive search request", scan.getRequestId())); + List rows = DataOperation.scan(scan.getTableInfo(), scan); + LOGGER.info(String.format("[QueryId:%d] sending success response", scan.getRequestId())); + return createSuccessResponse(scan, rows); + } catch (IOException e) { + LOGGER.error(e); + LOGGER.info(String.format("[QueryId:%d] sending failure response", scan.getRequestId())); + return createFailureResponse(scan, e); + } + } + + /** + * create a failure response + */ + private ScanResponse createFailureResponse(ScanRequest scan, Throwable throwable) { + return new ScanResponse(scan.getRequestId(), Status.FAILURE.ordinal(), + throwable.getMessage(), new Object[0][]); + } + + /** + * create a success response with result rows + */ + private ScanResponse createSuccessResponse(ScanRequest scan, List rows) { + Iterator itor = rows.iterator(); + Object[][] output = new Object[rows.size()][]; + int i = 0; + while (itor.hasNext()) { + output[i++] = itor.next().getData(); + } + return new ScanResponse(scan.getRequestId(), Status.SUCCESS.ordinal(), "", output); } @Override @@ -58,4 +164,11 @@ public ProtocolSignature getProtocolSignature(String protocol, long clientVersio return null; } + @Override + public void close() throws IOException { + LOGGER.info("Shutting down worker..."); + SearchModeDetailQueryExecutor.shutdownThreadPool(); + SearchModeVectorDetailQueryExecutor.shutdownThreadPool(); + LOGGER.info("Worker shut down"); + } } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java index b8a6f4478a9..0472b758bee 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStore.java @@ -94,12 +94,11 @@ public interface CarbonStore extends Closeable { * Return true if this table has primary key defined when create table using * {@link #createTable(TableDescriptor)} * - * For such table, {@link #newMutator(TableIdentifier)} and {@link #newFetcher(TableIdentifier)} - * are supported + * For such table, upsert, delete and lookup is supported * * @return true if this table has primary key. */ - default boolean isPrimaryKeyDefined() { + default boolean isPrimaryKeyDefined(TableIdentifier identifier) { return false; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java index 6f5bbb59723..fcbd88dd572 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/CarbonStoreFactory.java @@ -34,23 +34,23 @@ @InterfaceAudience.User @InterfaceStability.Unstable public class CarbonStoreFactory { - private static Map remoteStore = new ConcurrentHashMap<>(); - private static Map localStores = new ConcurrentHashMap<>(); + private static final Map distributedStore = new ConcurrentHashMap<>(); + private static final Map localStores = new ConcurrentHashMap<>(); private CarbonStoreFactory() { } - public static CarbonStore getDistributedStore(String storeName, StoreConf storeConf) - throws CarbonException { - if (remoteStore.containsKey(storeName)) { - return remoteStore.get(storeName); + public static synchronized CarbonStore getDistributedStore( + String storeName, StoreConf storeConf) throws CarbonException { + if (distributedStore.containsKey(storeName)) { + return distributedStore.get(storeName); } // create a new instance try { String className = "org.apache.carbondata.sdk.store.DistributedCarbonStore"; CarbonStore store = createCarbonStore(storeConf, className); - remoteStore.put(storeName, store); + distributedStore.put(storeName, store); return store; } catch (ClassNotFoundException | IllegalAccessException | InvocationTargetException | InstantiationException e) { @@ -58,14 +58,14 @@ public static CarbonStore getDistributedStore(String storeName, StoreConf storeC } } - public static void removeDistributedStore(String storeName) throws IOException { - if (remoteStore.containsKey(storeName)) { - remoteStore.get(storeName).close(); - remoteStore.remove(storeName); + public static synchronized void removeDistributedStore(String storeName) throws IOException { + if (distributedStore.containsKey(storeName)) { + distributedStore.get(storeName).close(); + distributedStore.remove(storeName); } } - public static CarbonStore getLocalStore(String storeName, StoreConf storeConf) + public static synchronized CarbonStore getLocalStore(String storeName, StoreConf storeConf) throws CarbonException { if (localStores.containsKey(storeName)) { return localStores.get(storeName); diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java index a7f0d6191dc..78349094bc2 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/DistributedCarbonStore.java @@ -26,7 +26,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory; import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.core.metadata.datatype.StructType; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; @@ -77,8 +77,8 @@ public List listTable() throws CarbonException { @Override public TableDescriptor getDescriptor(TableIdentifier table) throws CarbonException { - CarbonTable carbonTable = storeService.getTable(table); - // TODO: create TableDescriptor from carbonTable + TableInfo tableInfo = storeService.getTable(table); + // TODO: create TableDescriptor from table info return null; } diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java index 1b8ea53736b..f580fc5244e 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/descriptor/ScanDescriptor.java @@ -20,6 +20,7 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; +import java.io.Serializable; import java.util.Objects; import org.apache.carbondata.common.annotations.InterfaceAudience; @@ -31,7 +32,7 @@ @InterfaceAudience.User @InterfaceStability.Evolving -public class ScanDescriptor implements Writable { +public class ScanDescriptor implements Serializable, Writable { private TableIdentifier table; private String[] projection; diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java index fa5f0656c1f..7e54d2f5548 100644 --- a/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java +++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/store/service/StoreService.java @@ -21,7 +21,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.core.datastore.row.CarbonRow; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; +import org.apache.carbondata.core.metadata.schema.table.TableInfo; import org.apache.carbondata.sdk.store.descriptor.LoadDescriptor; import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; import org.apache.carbondata.sdk.store.descriptor.TableDescriptor; @@ -38,7 +38,7 @@ public interface StoreService extends VersionedProtocol { void dropTable(TableIdentifier table) throws CarbonException; - CarbonTable getTable(TableIdentifier table) throws CarbonException; + TableInfo getTable(TableIdentifier table) throws CarbonException; List listTable() throws CarbonException; diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java index d654590eec8..d0128fc263e 100644 --- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java +++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/AvroCarbonWriterTest.java @@ -29,14 +29,16 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.util.path.CarbonTablePath; -import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.commons.io.FileUtils; +import org.apache.commons.lang.CharEncoding; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.apache.avro.Schema; + public class AvroCarbonWriterTest { private String path = "./AvroCarbonWriterSuiteWriteFiles"; diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java index 75e4da9391d..19bf194c680 100644 --- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java +++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/CarbonReaderTest.java @@ -17,15 +17,12 @@ package org.apache.carbondata.sdk.file; -import java.io.File; -import java.io.FileFilter; -import java.io.FilenameFilter; -import java.io.IOException; +import java.io.*; import java.sql.Date; import java.sql.Timestamp; -import java.util.Arrays; -import java.util.Comparator; +import java.util.*; +import org.apache.avro.generic.GenericData; import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException; import org.apache.carbondata.common.logging.LogService; import org.apache.carbondata.common.logging.LogServiceFactory; @@ -33,25 +30,16 @@ import org.apache.carbondata.core.metadata.datatype.DataTypes; import org.apache.carbondata.core.scan.expression.ColumnExpression; import org.apache.carbondata.core.scan.expression.LiteralExpression; -import org.apache.carbondata.core.scan.expression.conditional.EqualToExpression; -import org.apache.carbondata.core.scan.expression.conditional.GreaterThanExpression; -import org.apache.carbondata.core.scan.expression.conditional.InExpression; -import org.apache.carbondata.core.scan.expression.conditional.LessThanExpression; -import org.apache.carbondata.core.scan.expression.conditional.NotEqualsExpression; -import org.apache.carbondata.core.scan.expression.conditional.NotInExpression; +import org.apache.carbondata.core.scan.expression.conditional.*; import org.apache.carbondata.core.scan.expression.logical.AndExpression; import org.apache.carbondata.core.scan.expression.logical.OrExpression; import org.apache.carbondata.core.util.CarbonProperties; import org.apache.carbondata.core.util.path.CarbonTablePath; import junit.framework.TestCase; -import org.apache.avro.generic.GenericData; import org.apache.commons.io.FileUtils; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import org.apache.commons.lang.CharEncoding; +import org.junit.*; public class CarbonReaderTest extends TestCase { From 9e575f080854973f2be8db325b99cd7a2eb0d00d Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Sun, 5 Aug 2018 01:26:20 +0800 Subject: [PATCH 13/14] add Interface tag --- .../java/org/apache/carbondata/store/devapi/DataLoader.java | 2 +- .../java/org/apache/carbondata/store/devapi/DataScanner.java | 4 ++++ .../carbondata/store/devapi/InternalCarbonStoreFactory.java | 4 ++++ .../main/java/org/apache/carbondata/store/devapi/Pruner.java | 4 ++++ .../java/org/apache/carbondata/store/devapi/ResultBatch.java | 2 +- .../java/org/apache/carbondata/store/devapi/ScanOption.java | 2 +- .../java/org/apache/carbondata/store/devapi/ScanUnit.java | 2 +- .../main/java/org/apache/carbondata/store/devapi/Scanner.java | 2 +- .../carbondata/store/devapi/TransactionalOperation.java | 2 +- .../carbondata/store/impl/master/RegistryServiceImpl.java | 2 +- 10 files changed, 19 insertions(+), 7 deletions(-) diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java index 7e8aefac863..4b79ee4d611 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataLoader.java @@ -29,7 +29,7 @@ /** * A Loader is used to load data from files to the table */ -@InterfaceAudience.User +@InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable public interface DataLoader extends TransactionalOperation, Serializable { /** diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java index 58a90dc55eb..ef46cf74e8d 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java @@ -19,8 +19,12 @@ import java.util.Iterator; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.sdk.store.exception.CarbonException; +@InterfaceAudience.Developer("Integration") +@InterfaceStability.Unstable public interface DataScanner { /** diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java index 6a4644426b7..c875aa1f51c 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/InternalCarbonStoreFactory.java @@ -21,9 +21,13 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.sdk.store.conf.StoreConf; import org.apache.carbondata.store.impl.InternalCarbonStoreImpl; +@InterfaceAudience.Developer("Integration") +@InterfaceStability.Unstable public class InternalCarbonStoreFactory { private static final Map stores = new ConcurrentHashMap<>(); diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java index cc15865a7e0..4a1d2e535f6 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/Pruner.java @@ -19,10 +19,14 @@ import java.util.List; +import org.apache.carbondata.common.annotations.InterfaceAudience; +import org.apache.carbondata.common.annotations.InterfaceStability; import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; +@InterfaceAudience.Developer("Integration") +@InterfaceStability.Unstable public interface Pruner { /** diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java index 65da069c9b3..221e0f66c14 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ResultBatch.java @@ -20,7 +20,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -@InterfaceAudience.User +@InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable public interface ResultBatch { diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java index 795a2db5b2a..8e5797d8642 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanOption.java @@ -22,7 +22,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -@InterfaceAudience.User +@InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable public class ScanOption { diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java index 65db868e1fd..fdcffdfe5b1 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/ScanUnit.java @@ -26,7 +26,7 @@ /** * An unit for the scanner in Carbon Store */ -@InterfaceAudience.User +@InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable public interface ScanUnit extends Serializable, Writable { diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java index 217be1346d6..e56eee218c8 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/Scanner.java @@ -26,7 +26,7 @@ * A Scanner is used to scan the table in a distributed compute * engine like Apache Spark */ -@InterfaceAudience.User +@InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable public interface Scanner extends DataScanner, Pruner, Serializable { diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java index 9c7249b009b..29b3bce0683 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/TransactionalOperation.java @@ -20,7 +20,7 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; -@InterfaceAudience.User +@InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable public interface TransactionalOperation { /** diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java index 73520f6f060..ce2be99d0e0 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/master/RegistryServiceImpl.java @@ -27,7 +27,7 @@ import org.apache.hadoop.ipc.ProtocolSignature; @InterfaceAudience.Internal -class RegistryServiceImpl implements RegistryService { +class egistryServiceImpl implements RegistryService { private Master master; From 1f387e06a355abd06d83306ef27bc81413439885 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 6 Aug 2018 01:49:09 +0800 Subject: [PATCH 14/14] fix comment --- .../carbondata/store/devapi/DataScanner.java | 5 +- .../store/impl/DelegatedScanner.java | 57 +++++++++ .../store/impl/InternalCarbonStoreImpl.java | 30 ++--- .../store/impl/LocalDataScanner.java | 4 + .../store/impl/RemoteDataScanner.java | 6 +- .../carbondata/store/impl/RemoteScanner.java | 117 ------------------ .../store/impl/service/DataService.java | 4 + .../store/impl/service/PruneService.java | 10 ++ 8 files changed, 93 insertions(+), 140 deletions(-) create mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/DelegatedScanner.java delete mode 100644 store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java diff --git a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java index ef46cf74e8d..c6c56285e30 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/devapi/DataScanner.java @@ -17,15 +17,18 @@ package org.apache.carbondata.store.devapi; +import java.io.Serializable; import java.util.Iterator; import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.common.annotations.InterfaceStability; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; import org.apache.carbondata.sdk.store.exception.CarbonException; @InterfaceAudience.Developer("Integration") @InterfaceStability.Unstable -public interface DataScanner { +public interface DataScanner extends Serializable { /** * Perform a scan in a distributed compute framework like Spark, Presto, etc. diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/DelegatedScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/DelegatedScanner.java new file mode 100644 index 00000000000..bd406672747 --- /dev/null +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/DelegatedScanner.java @@ -0,0 +1,57 @@ +/* + * 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.carbondata.store.impl; + +import java.util.Iterator; +import java.util.List; + +import org.apache.carbondata.common.logging.LogService; +import org.apache.carbondata.common.logging.LogServiceFactory; +import org.apache.carbondata.core.scan.expression.Expression; +import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; +import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.DataScanner; +import org.apache.carbondata.store.devapi.Pruner; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanUnit; +import org.apache.carbondata.store.devapi.Scanner; + +public class DelegatedScanner implements Scanner { + private static final LogService LOGGER = + LogServiceFactory.getLogService(DelegatedScanner.class.getCanonicalName()); + + private Pruner pruner; + private DataScanner scanner; + + public DelegatedScanner(Pruner pruner, DataScanner scanner) { + this.pruner = pruner; + this.scanner = scanner; + } + + @Override + public List prune(TableIdentifier table, Expression filterExpression) + throws CarbonException { + return pruner.prune(table, filterExpression); + } + + @Override + public Iterator> scan(ScanUnit input) throws CarbonException { + return scanner.scan(input); + } + +} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java b/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java index bb3f3fc10e4..4821116d9c0 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/InternalCarbonStoreImpl.java @@ -18,8 +18,6 @@ package org.apache.carbondata.store.impl; import java.io.IOException; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.ConcurrentHashMap; @@ -27,7 +25,6 @@ import org.apache.carbondata.common.annotations.InterfaceAudience; import org.apache.carbondata.core.metadata.schema.table.CarbonTable; import org.apache.carbondata.core.metadata.schema.table.TableInfo; -import org.apache.carbondata.core.scan.expression.Expression; import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; import org.apache.carbondata.sdk.store.DistributedCarbonStore; import org.apache.carbondata.sdk.store.conf.StoreConf; @@ -39,9 +36,7 @@ import org.apache.carbondata.store.devapi.DataScanner; import org.apache.carbondata.store.devapi.InternalCarbonStore; import org.apache.carbondata.store.devapi.Pruner; -import org.apache.carbondata.store.devapi.ResultBatch; import org.apache.carbondata.store.devapi.ScanOption; -import org.apache.carbondata.store.devapi.ScanUnit; import org.apache.carbondata.store.devapi.Scanner; /** @@ -84,14 +79,17 @@ public Scanner newScanner(TableIdentifier identifier, ScanDescriptor scan throws CarbonException { Objects.requireNonNull(identifier); Objects.requireNonNull(scanDescriptor); + boolean isRemotePrune; + boolean isOpPushdown; if (scanOption == null) { - return new RemoteScanner<>(storeConf, getCarbonTable(identifier), scanDescriptor, scanOption, - readSupport); + isRemotePrune = true; + isOpPushdown = true; + } else { + isRemotePrune = ScanOption.isRemotePrune(scanOption); + isOpPushdown = ScanOption.isOperatorPushdown(scanOption); } - TableInfo tableInfo = MetaOperation.getTable(identifier, storeConf); - boolean isRemotePrune = ScanOption.isRemotePrune(scanOption); - boolean isOpPushdown = ScanOption.isOperatorPushdown(scanOption); + TableInfo tableInfo = MetaOperation.getTable(identifier, storeConf); Pruner pruner; DataScanner scanner; @@ -106,17 +104,7 @@ public Scanner newScanner(TableIdentifier identifier, ScanDescriptor scan scanner = new LocalDataScanner<>(storeConf, scanDescriptor, scanOption); } - return new Scanner() { - @Override public List prune(TableIdentifier table, Expression filterExpression) - throws CarbonException { - return pruner.prune(table, filterExpression); - } - - @Override public Iterator> scan(ScanUnit input) - throws CarbonException { - return scanner.scan(input); - } - }; + return new DelegatedScanner<>(pruner, scanner); } } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java index ed329c22c30..c3429a4920e 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/LocalDataScanner.java @@ -35,6 +35,10 @@ import org.apache.carbondata.store.devapi.ScanUnit; import org.apache.carbondata.store.impl.service.model.ScanRequest; +/** + * This scanner scans in local JVM + * @param scan output + */ public class LocalDataScanner implements DataScanner { private StoreConf storeConf; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java index 8c5edf22641..81254266f0f 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteDataScanner.java @@ -40,6 +40,10 @@ import org.apache.carbondata.store.impl.service.model.ScanRequest; import org.apache.carbondata.store.impl.service.model.ScanResponse; +/** + * This scanner scans in local JVM + * @param scan output + */ public class RemoteDataScanner implements DataScanner { private TableInfo tableInfo; @@ -75,7 +79,7 @@ public Iterator> scan(ScanUnit input) throws CarbonExce .map(row -> readSupport.readRow(row)) .collect(Collectors.toList()); - return Collections.singletonList(new RowMajorResultBatch(rows)).iterator(); + return Collections.singletonList(new RowMajorResultBatch<>(rows)).iterator(); } catch (IOException e) { throw new CarbonException(e); } diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java b/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java deleted file mode 100644 index d8035add353..00000000000 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/RemoteScanner.java +++ /dev/null @@ -1,117 +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.carbondata.store.impl; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.stream.Collectors; - -import org.apache.carbondata.common.logging.LogService; -import org.apache.carbondata.common.logging.LogServiceFactory; -import org.apache.carbondata.core.metadata.schema.table.CarbonTable; -import org.apache.carbondata.core.metadata.schema.table.TableInfo; -import org.apache.carbondata.core.scan.expression.Expression; -import org.apache.carbondata.hadoop.CarbonInputSplit; -import org.apache.carbondata.hadoop.CarbonMultiBlockSplit; -import org.apache.carbondata.hadoop.readsupport.CarbonReadSupport; -import org.apache.carbondata.sdk.store.conf.StoreConf; -import org.apache.carbondata.sdk.store.descriptor.ScanDescriptor; -import org.apache.carbondata.sdk.store.descriptor.TableIdentifier; -import org.apache.carbondata.sdk.store.exception.CarbonException; -import org.apache.carbondata.store.devapi.ResultBatch; -import org.apache.carbondata.store.devapi.ScanUnit; -import org.apache.carbondata.store.devapi.Scanner; -import org.apache.carbondata.store.impl.service.DataService; -import org.apache.carbondata.store.impl.service.PruneService; -import org.apache.carbondata.store.impl.service.ServiceFactory; -import org.apache.carbondata.store.impl.service.model.PruneRequest; -import org.apache.carbondata.store.impl.service.model.PruneResponse; -import org.apache.carbondata.store.impl.service.model.ScanRequest; -import org.apache.carbondata.store.impl.service.model.ScanResponse; - -/** - * This Scanner executes pruning and scanning in remote Master and Worker - */ -public class RemoteScanner implements Scanner { - private static final LogService LOGGER = - LogServiceFactory.getLogService(RemoteScanner.class.getCanonicalName()); - - private TableInfo tableInfo; - private String pruneServiceHost; - private int pruneServiePort; - private ScanDescriptor scanDescriptor; - private Map scanOption; - private CarbonReadSupport readSupport; - - RemoteScanner(StoreConf conf, CarbonTable carbonTable, ScanDescriptor scanDescriptor, - Map scanOption, CarbonReadSupport readSupport) { - this.tableInfo = carbonTable.getTableInfo(); - this.pruneServiceHost = conf.masterHost(); - this.pruneServiePort = conf.pruneServicePort(); - this.scanDescriptor = scanDescriptor; - this.scanOption = scanOption; - this.readSupport = readSupport; - } - - @Override - public List prune(TableIdentifier table, Expression filterExpression) - throws CarbonException { - try { - PruneRequest request = new PruneRequest(table, filterExpression); - PruneService pruneService = ServiceFactory.createPruneService( - pruneServiceHost, pruneServiePort); - PruneResponse response = pruneService.prune(request); - return response.getScanUnits(); - } catch (IOException e) { - throw new CarbonException(e); - } - } - - @Override - public Iterator> scan(ScanUnit input) throws CarbonException { - List toBeScan = new ArrayList<>(); - if (input instanceof BlockScanUnit) { - toBeScan.add(((BlockScanUnit) input).getInputSplit()); - } else { - throw new CarbonException(input.getClass().getName() + " is not supported"); - } - int queryId = new Random().nextInt(); - CarbonMultiBlockSplit split = new CarbonMultiBlockSplit(toBeScan, input.preferredLocations()); - try { - ScanRequest request = new ScanRequest(queryId, split, tableInfo, - scanDescriptor.getProjection(), scanDescriptor.getFilter(), scanDescriptor.getLimit()); - DataService dataService = - DataServicePool.getOrCreateDataService(((BlockScanUnit) input).getSchedulable()); - ScanResponse response = dataService.scan(request); - List rows = Arrays.stream(response.getRows()) - .map(row -> readSupport.readRow(row)) - .collect(Collectors.toList()); - - return Collections.singletonList(new RowMajorResultBatch(rows)).iterator(); - } catch (IOException e) { - throw new CarbonException(e); - } - } - -} diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java index de28252a48c..670540b5b11 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/DataService.java @@ -18,8 +18,12 @@ package org.apache.carbondata.store.impl.service; import java.io.Closeable; +import java.util.Iterator; +import org.apache.carbondata.core.datastore.row.CarbonRow; import org.apache.carbondata.sdk.store.exception.CarbonException; +import org.apache.carbondata.store.devapi.ResultBatch; +import org.apache.carbondata.store.devapi.ScanUnit; import org.apache.carbondata.store.impl.service.model.BaseResponse; import org.apache.carbondata.store.impl.service.model.LoadDataRequest; import org.apache.carbondata.store.impl.service.model.ScanRequest; diff --git a/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java b/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java index 1f6306e2ac4..d69db9639ab 100644 --- a/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java +++ b/store/core/src/main/java/org/apache/carbondata/store/impl/service/PruneService.java @@ -27,5 +27,15 @@ @InterfaceAudience.Internal public interface PruneService extends VersionedProtocol { long versionID = 1L; + + /** + * Leveraging index and segment information to skip blocks, + * return a list of block eligible for scanning + * + * @param request prune request containing table identifier + * and filter expression + * @return prune result containing blocks to scan + * @throws CarbonException if any error occurs + */ PruneResponse prune(PruneRequest request) throws CarbonException; }