From 2972c4dc04880ad141b253e85388e9c2594cda44 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Wed, 30 May 2018 05:42:40 +0530 Subject: [PATCH 01/19] Update store link --- .../scala/io/snappydata/cluster/SnappySecureJob.scala | 2 +- cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala | 8 +++++--- .../sql/execution/columnar/impl/ColumnFormatEntry.scala | 4 ++++ store | 2 +- 4 files changed, 11 insertions(+), 5 deletions(-) diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala index bbfe14fae7..03f6c2b9ec 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SnappySecureJob.scala @@ -60,7 +60,7 @@ class SnappySecureJob extends SnappySQLJob { accessAndModifyTablesOwnedByOthers(snSession, jobConfig) } // Confirm that our zeppelin interpreter is not initialized. - assert(ServiceManager.getLeadInstance.asInstanceOf[LeadImpl].getInterpreterServerClass() == + assert(ServiceManager.getLeadInstance.asInstanceOf[LeadImpl].getInterpreterServerClass == null, "Zeppelin interpreter must not be initialized in secure cluster") // Check SnappyData Pulse UI is secured by our custom authenticator. assert(SparkCallbacks.getAuthenticatorForJettyServer().get diff --git a/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala b/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala index 2bc2a9d30a..6825d4850c 100644 --- a/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala +++ b/cluster/src/main/scala/io/snappydata/impl/LeadImpl.scala @@ -16,7 +16,7 @@ */ package io.snappydata.impl -import java.lang.reflect.{Constructor, InvocationTargetException, Method} +import java.lang.reflect.{Constructor, Method} import java.net.{URL, URLClassLoader} import java.security.Permission import java.sql.SQLException @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} + import akka.actor.ActorSystem import com.gemstone.gemfire.CancelException import com.gemstone.gemfire.cache.CacheClosedException @@ -50,6 +51,7 @@ import org.apache.thrift.transport.TTransportException import spark.jobserver.JobServer import spark.jobserver.auth.{AuthInfo, SnappyAuthenticator, User} import spray.routing.authentication.UserPass + import org.apache.spark.sql.collection.{ToolsCallbackInit, Utils} import org.apache.spark.sql.{SnappyContext, SnappySession} import org.apache.spark.{Logging, SparkCallbacks, SparkConf, SparkContext, SparkException} @@ -688,11 +690,11 @@ class LeadImpl extends ServerImpl with Lead } finally { System.setSecurityManager(origSecurityManager) } - checkAndStartZeppelinInterpreter(true, bootProperties) + checkAndStartZeppelinInterpreter(enabled = true, bootProperties) } } - def getInterpreterServerClass(): Class[_] = { + def getInterpreterServerClass: Class[_] = { remoteInterpreterServerClass } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala index 4d863522cd..de241e813c 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala @@ -93,6 +93,10 @@ object ColumnFormatEntry { /** * Key object in the column store. + * + * @param uuid an ID for the key which should be unique in the cluster for a region + * @param partitionId the bucket ID of the key; must be same as ID of bucket where key is put + * @param columnIndex 1-based column index for the key (negative for meta-data and delta columns) */ final class ColumnFormatKey(private[columnar] var uuid: Long, private[columnar] var partitionId: Int, diff --git a/store b/store index e806f70d70..7b6450d26c 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit e806f70d704facd1d09533dc54644b19d73ea54f +Subproject commit 7b6450d26c514d4f556a4092d778444a6177669d From ff3cca07352fba869019b2b12527d52fc7ddf34f Mon Sep 17 00:00:00 2001 From: Trilok Khairnar Date: Wed, 30 May 2018 23:04:21 +0530 Subject: [PATCH 02/19] fix for SNAP-2365 --- .../TPCHColumnPartitionedTable.scala | 60 +++++++++++++------ .../benchmark/snappy/tpch/SparkApp.scala | 12 ++-- .../snappy/tpch/SparkAppUsingJob.scala | 10 ++-- 3 files changed, 53 insertions(+), 29 deletions(-) diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala index 83270bd8b6..948ccf53d0 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala @@ -121,18 +121,22 @@ object TPCHColumnPartitionedTable { def createPopulateOrderTable(sqlContext: SQLContext, path: String, isSnappy: Boolean, buckets: String = "128", loadPerfPrintStream: PrintStream = null, redundancy : String = "0", - persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStage : Int = 1, + persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStages : Int = 1, isParquet : Boolean = false) : Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var orderDF: DataFrame = null var unionOrderDF: DataFrame = null // use parquet data if available - for (i <- 1 to numberOfLoadingStage) { + for (i <- 1 to numberOfLoadingStages) { if (isParquet) { orderDF = sqlContext.read.format("parquet").load(s"$path/parquet_orders_$i") } else { - val orderData = sc.textFile(s"$path/orders.tbl.$i") + var stage = ""; // apply a tbl.i suffix to table filename only when data is loaded in more than one stages. + if (numberOfLoadingStages > 1) { + stage = s".$i"; + } + val orderData = sc.textFile(s"$path/orders.tbl$stage") val orderReadings = orderData.map(s => s.split('|')).map( s => TPCHTableSchema.parseOrderRow(s)) val orderDF1 = sqlContext.createDataFrame(orderReadings) @@ -206,18 +210,22 @@ object TPCHColumnPartitionedTable { def createPopulateLineItemTable(sqlContext: SQLContext, path: String, isSnappy: Boolean, buckets: String = "128", loadPerfPrintStream: PrintStream = null, redundancy : String = "0", - persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStage : Int = 1, + persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStages : Int = 1, isParquet : Boolean = false) : Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var lineItemDF: DataFrame = null var unionLineItemDF: DataFrame = null // use parquet data if available - for (i <- 1 to numberOfLoadingStage) { + for (i <- 1 to numberOfLoadingStages) { if (isParquet) { lineItemDF = sqlContext.read.format("parquet").load(s"$path/parquet_lineitem_$i") } else { - val lineItemData = sc.textFile(s"$path/lineitem.tbl.$i") + var stage = ""; + if (numberOfLoadingStages > 1) { + stage = s".$i"; + } + val lineItemData = sc.textFile(s"$path/lineitem.tbl$stage") val lineItemReadings = lineItemData.map(s => s.split('|')).map(s => TPCHTableSchema .parseLineItemRow(s)) val lineItemDF1 = sqlContext.createDataFrame(lineItemReadings) @@ -292,18 +300,22 @@ object TPCHColumnPartitionedTable { def createPopulateCustomerTable(sqlContext: SQLContext, path: String, isSnappy: Boolean, buckets: String = "128", loadPerfPrintStream: PrintStream = null, redundancy : String = "0", - persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStage : Int = 1, + persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStages : Int = 1, isParquet : Boolean = false) : Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var customerDF: DataFrame = null var unionCustomerDF: DataFrame = null - for (i <- 1 to numberOfLoadingStage) { + for (i <- 1 to numberOfLoadingStages) { // use parquet data if available if (isParquet) { customerDF = sqlContext.read.format("parquet").load(s"$path/parquet_customer_$i") } else { - val customerData = sc.textFile(s"$path/customer.tbl.$i") + var stage = ""; + if (numberOfLoadingStages > 1) { + stage = s".$i"; + } + val customerData = sc.textFile(s"$path/customer.tbl$stage") val customerReadings = customerData.map(s => s.split('|')).map(s => TPCHTableSchema .parseCustomerRow(s)) val customerDF1 = sqlContext.createDataFrame(customerReadings) @@ -355,18 +367,22 @@ object TPCHColumnPartitionedTable { def createPopulatePartTable(sqlContext: SQLContext, path: String, isSnappy: Boolean, buckets: String = "128", loadPerfPrintStream: PrintStream = null, redundancy : String = "0", - persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStage : Int = 1, + persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStages : Int = 1, isParquet : Boolean = false) : Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var partDF: DataFrame = null var unionPartDF: DataFrame = null - for(i <- 1 to numberOfLoadingStage) { + for(i <- 1 to numberOfLoadingStages) { // use parquet data if available if (isParquet) { partDF = sqlContext.read.format("parquet").load(s"$path/parquet_part_$i") } else { - val partData = sc.textFile(s"$path/part.tbl.$i") + var stage = ""; + if (numberOfLoadingStages > 1) { + stage = s".$i"; + } + val partData = sc.textFile(s"$path/part.tbl$stage") val partReadings = partData.map(s => s.split('|')).map(s => TPCHTableSchema.parsePartRow(s)) val partDF1 = sqlContext.createDataFrame(partReadings) val newSchema = TPCHTableSchema.newPartSchema(partDF1.schema) @@ -415,18 +431,22 @@ object TPCHColumnPartitionedTable { def createPopulatePartSuppTable(sqlContext: SQLContext, path: String, isSnappy: Boolean, buckets: String = "128", loadPerfPrintStream: PrintStream = null, redundancy : String = "0", - persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStage : Int = 1, + persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStages : Int = 1, isParquet : Boolean = false) : Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var partSuppDF: DataFrame = null var unionPartSuppDF: DataFrame = null - for (i <- 1 to numberOfLoadingStage) { + for (i <- 1 to numberOfLoadingStages) { // use parquet data if available if (isParquet) { partSuppDF = sqlContext.read.format("parquet").load(s"$path/parquet_partsupp_$i") } else { - val partSuppData = sc.textFile(s"$path/partsupp.tbl.$i") + var stage = ""; + if (numberOfLoadingStages > 1) { + stage = s".$i"; + } + val partSuppData = sc.textFile(s"$path/partsupp.tbl$stage") val partSuppReadings = partSuppData.map(s => s.split('|')).map(s => TPCHTableSchema .parsePartSuppRow(s)) val partSuppDF1 = sqlContext.createDataFrame(partSuppReadings) @@ -554,20 +574,24 @@ object TPCHColumnPartitionedTable { def createAndPopulateSupplierTable(sqlContext: SQLContext, path: String, isSnappy: Boolean, buckets: String = "128", loadPerfPrintStream: PrintStream = null, redundancy : String = "0", - persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStage : Int = 1, + persistence: Boolean = false, persistence_type: String = "", numberOfLoadingStages : Int = 1, isParquet : Boolean = false): Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var suppDF: DataFrame = null var unionSuppDF: DataFrame = null - for (i <- 1 to numberOfLoadingStage) { + for (i <- 1 to numberOfLoadingStages) { // use parquet data if available val parquetDir = s"$path/parquet_supplier_$i" if (isParquet && new File(parquetDir).exists()) { suppDF = sqlContext.read.format("parquet").load(parquetDir) } else { - val suppData = sc.textFile(s"$path/supplier.tbl.$i") + var stage = ""; + if (numberOfLoadingStages > 1) { + stage = s".$i"; + } + val suppData = sc.textFile(s"$path/supplier.tbl$stage") val suppReadings = suppData.map(s => s.split('|')).map(s => TPCHTableSchema .parseSupplierRow(s)) val suppDF1 = sqlContext.createDataFrame(suppReadings) diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala index 8645901a3d..4fa97f1b19 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkApp.scala @@ -66,23 +66,23 @@ object SparkApp { TPCHColumnPartitionedTable.createPopulateOrderTable(sc.sqlContext, tpchDataPath, false, if (rePartition) buckets_Order_Lineitem else "0", - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulateLineItemTable(sc.sqlContext, tpchDataPath, false, if (rePartition) buckets_Order_Lineitem else "0", - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulateCustomerTable(sc.sqlContext, tpchDataPath, false, if (rePartition) buckets_Cust_Part_PartSupp else "0", - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulatePartTable(sc.sqlContext, tpchDataPath, false, if (rePartition) buckets_Cust_Part_PartSupp else "0", - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulatePartSuppTable(sc.sqlContext, tpchDataPath, false, if (rePartition) buckets_Cust_Part_PartSupp else "0", - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHReplicatedTable.createPopulateRegionTable(usingOptionString, sc.sqlContext, tpchDataPath, @@ -92,7 +92,7 @@ object SparkApp { if (isSupplierColumn) { TPCHColumnPartitionedTable.createAndPopulateSupplierTable(sc.sqlContext, tpchDataPath, false, if (rePartition) buckets_Supplier else "0", - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) } else { TPCHReplicatedTable.createPopulateSupplierTable(usingOptionString, sc.sqlContext, diff --git a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala index 5e76914f7b..22a360d4b1 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/snappy/tpch/SparkAppUsingJob.scala @@ -65,19 +65,19 @@ object SparkAppUsingJob extends SnappySQLJob { TPCHColumnPartitionedTable.createPopulateOrderTable(snc, tpchDataPath, isSnappy, - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulateLineItemTable(snc, tpchDataPath, isSnappy, - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulateCustomerTable(snc, tpchDataPath, isSnappy, - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulatePartTable(snc, tpchDataPath, isSnappy, - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHColumnPartitionedTable.createPopulatePartSuppTable(snc, tpchDataPath, isSnappy, - loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStage = numberOfLoadStages, + loadPerfPrintStream = loadPerfPrintStream, numberOfLoadingStages = numberOfLoadStages, isParquet = isParquet) TPCHReplicatedTable.createPopulateRegionTable(usingOptionString, snc, tpchDataPath, From 608bf3dbd8e503aa6c464a9b03fe8d35cda61c8c Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 31 May 2018 18:15:32 +0530 Subject: [PATCH 03/19] [SNAP-2366] row buffer fault-in, forced rollover, merge small batches - add check for the two cases in table stats service: - "large enough" row buffer (currently "large enough" is anything more than maxDeltaRows/8) that has not seen any updates/deletes since the last check; in this case schedule a task to force rollover the row buffer in column table - also check if a bucket of column table has multiple small batches (non-transactional check); if so then submit a task to merge those after checking for transactional snapshot; merge is done by locally created ColumnTableScan->ColumnInsertExec plan where the scan uses an iterator only on the small batches - added a ColumnFormatStatsIterator that can take a bunch of stats rows and create an iterator over just those (like required for batch merge) - added new scan metrics for disk reads: a) disk rows from row buffer, b) partial column batches on disk, c) full column batches on disk - extended SQLMetrics types with a new SPLIT_SUM_METRIC that allows displaying multiple metrics against a common name; ColumnTableScan now uses this to combine some metrics else it becomes too large in display (especially for the newly added disk read metrics) - use hive-metadata (ExternalTableMetaData) to get number of rows instead of getting from row buffer table (that is subject to changes in future) --- .../SnappyTableStatsProviderService.scala | 302 +++++++++++++++--- ...nappyThinConnectorTableStatsProvider.scala | 6 +- .../TableStatsProviderService.scala | 15 +- .../apache/spark/sql/CachedDataFrame.scala | 4 +- .../apache/spark/sql/collection/Utils.scala | 15 +- .../spark/sql/execution/SnappyMetrics.scala | 67 ++++ .../spark/sql/execution/TableExec.scala | 15 + .../sql/execution/columnar/ColumnBatch.scala | 62 ++-- .../columnar/ColumnBatchCreator.scala | 2 +- .../execution/columnar/ColumnInsertExec.scala | 4 +- .../execution/columnar/ColumnTableScan.scala | 57 +++- .../execution/columnar/impl/ColumnDelta.scala | 4 +- .../columnar/impl/ColumnFormatEncoder.scala | 6 +- .../columnar/impl/ColumnFormatEntry.scala | 26 +- .../columnar/impl/ColumnFormatIterator.scala | 37 +++ .../impl/JDBCSourceAsColumnarStore.scala | 10 +- .../sql/execution/row/RowFormatScanRDD.scala | 45 ++- .../sql/execution/ui/SnappySQLListener.scala | 95 +++++- .../spark/sql/store/CompressionCodecId.scala | 8 +- .../apache/spark/sql/store/StoreUtils.scala | 8 +- store | 2 +- 21 files changed, 666 insertions(+), 124 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index 0a86a7b2fc..44af11a074 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -19,29 +19,39 @@ package io.snappydata +import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ConcurrentHashMap, TimeUnit} -import java.util.function.BiFunction +import java.util.function.{BiFunction, Predicate} import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.Future import scala.language.implicitConversions import scala.util.control.NonFatal -import com.gemstone.gemfire.CancelException +import com.gemstone.gemfire.cache.IsolationLevel import com.gemstone.gemfire.cache.execute.FunctionService import com.gemstone.gemfire.i18n.LogWriterI18n import com.gemstone.gemfire.internal.SystemTimer -import com.gemstone.gemfire.internal.cache.{AbstractRegionEntry, ExternalTableMetaData, LocalRegion, PartitionedRegion, RegionEntry} +import com.gemstone.gemfire.internal.cache._ +import com.gemstone.gemfire.{CancelException, SystemFailure} import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.distributed.GfxdListResultCollector.ListResultCollectorValue import com.pivotal.gemfirexd.internal.engine.distributed.{GfxdListResultCollector, GfxdMessage} import com.pivotal.gemfirexd.internal.engine.sql.execute.MemberStatisticsMessage import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer import com.pivotal.gemfirexd.internal.engine.ui._ -import io.snappydata.Constant._ import io.snappydata.collection.ObjectObjectHashMap import org.apache.spark.SparkContext -import org.apache.spark.sql.execution.columnar.impl.{ColumnFormatKey, ColumnFormatRelation, ColumnFormatValue, RemoteEntriesIterator} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.collection.Utils +import org.apache.spark.sql.execution.columnar.impl.{ColumnDelta, ColumnFormatEntry, ColumnFormatKey, ColumnFormatValue, RemoteEntriesIterator} +import org.apache.spark.sql.execution.columnar.{ColumnBatchIterator, ColumnInsertExec, ColumnTableScan, ExternalStore, ExternalStoreUtils} +import org.apache.spark.sql.execution.row.ResultSetTraversal +import org.apache.spark.sql.execution.{BufferedRowIterator, WholeStageCodegenExec} +import org.apache.spark.sql.store.CodeGeneration +import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{SnappyContext, ThinClientConnectorMode} /* @@ -86,8 +96,6 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService if (!doRun) { this.synchronized { if (!doRun) { - val delay = sc.getConf.getLong(SPARK_SNAPPY_PREFIX + - "calcTableSizeInterval", DEFAULT_CALC_TABLE_SIZE_SERVICE_INTERVAL) doRun = true Misc.getGemFireCache.getCCPTimer.schedule( new SystemTimer.SystemTimerTask { @@ -112,8 +120,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService override def getLoggerI18n: LogWriterI18n = { logger } - }, - delay, delay) + }, delayMillis, delayMillis) } } } @@ -164,8 +171,8 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService override def getStatsFromAllServers(sc: Option[SparkContext] = None): (Seq[SnappyRegionStats], Seq[SnappyIndexStats], Seq[SnappyExternalTableStats]) = { - var result = new java.util.ArrayList[SnappyRegionStatsCollectorResult]().asScala - var externalTables = scala.collection.mutable.Buffer.empty[SnappyExternalTableStats] + var result: Seq[SnappyRegionStatsCollectorResult] = Nil + var externalTables: Seq[SnappyExternalTableStats] = Nil val dataServers = GfxdMessage.getAllDataStores try { if (dataServers != null && dataServers.size() > 0) { @@ -194,16 +201,13 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService case NonFatal(e) => log.warn(e.getMessage, e) } - if(result.flatMap(_.getRegionStats.asScala).size == 0) { + val tableStats = result.flatMap(_.getRegionStats.asScala) + if (tableStats.isEmpty) { // Return last updated tableSizeInfo - (tableSizeInfo.values.toSeq, - result.flatMap(_.getIndexStats.asScala), - externalTables) + (tableSizeInfo.values.toSeq, result.flatMap(_.getIndexStats.asScala), externalTables) } else { // Return updated tableSizeInfo - (result.flatMap(_.getRegionStats.asScala), - result.flatMap(_.getIndexStats.asScala), - externalTables) + (tableStats, result.flatMap(_.getIndexStats.asScala), externalTables) } } @@ -215,55 +219,271 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService */ private val createRemoteIterator = new BiFunction[java.lang.Integer, PRIterator, java.util.Iterator[RegionEntry]] { - override def apply(bucketId: Integer, - iter: PRIterator): java.util.Iterator[RegionEntry] = { - new RemoteEntriesIterator(bucketId, Array.emptyIntArray, - iter.getPartitionedRegion, null) + override def apply(bucketId: Integer, iter: PRIterator): java.util.Iterator[RegionEntry] = { + new RemoteEntriesIterator(bucketId, Array.emptyIntArray, iter.getPartitionedRegion, null) + } + } + + private def handleException(t: Throwable): Unit = t match { + case e: Error if SystemFailure.isJVMFailureError(e) => + SystemFailure.initiateFailure(e) + // If this ever returns, rethrow the error. We're poisoned + // now, so don't let this thread continue. + throw e + case _ => + // Whenever you catch Error or Throwable, you must also + // check for fatal JVM error (see above). However, there is + // _still_ a possibility that you are dealing with a cascading + // error condition, so you also need to check to see if the JVM + // is still usable: + SystemFailure.checkFailure() + logWarning(t.getMessage, t) + throw t + } + + private def handleTransaction(cache: GemFireCacheImpl, tx: TXStateProxy, + context: TXManagerImpl.TXContext, success: Boolean): Unit = { + if (tx ne null) { + val txManager = cache.getCacheTransactionManager + if (success) { + txManager.commit(tx, null, TXManagerImpl.FULL_COMMIT, context, false) + if (cache.getRvvSnapshotTestHook ne null) { + cache.notifyRvvTestHook() + cache.waitOnRvvSnapshotTestHook() + } + } else { + txManager.rollback(tx, null, false) + } + } + } + + private def withExceptionHandling(f: => Unit): Unit = { + try { + f + } catch { + case t: Throwable => handleException(t) } } def publishColumnTableRowCountStats(): Unit = { def asSerializable[C](c: C) = c.asInstanceOf[C with Serializable] - val regions = asSerializable(Misc.getGemFireCache.getApplicationRegions.asScala) - for (region: LocalRegion <- regions) { + val cache = Misc.getGemFireCache + val regions = asSerializable(cache.getApplicationRegions.asScala) + // Transaction started to check for committed entries if required. + val txManager = cache.getCacheTransactionManager + var context: TXManagerImpl.TXContext = null + val tx = if (cache.snapshotEnabled) { + context = TXManagerImpl.getOrCreateTXContext() + txManager.beginTX(context, IsolationLevel.SNAPSHOT, null, null) + } else null + var success = true + try for (region: LocalRegion <- regions) { if (region.getDataPolicy.withPartitioning()) { - val table = Misc.getFullTableNameFromRegionPath(region.getFullPath) val pr = region.asInstanceOf[PartitionedRegion] val container = pr.getUserAttribute.asInstanceOf[GemFireContainer] - if (ColumnFormatRelation.isColumnTable(table) && - pr.getLocalMaxMemory > 0) { - var numColumnsInTable = -1 + val isColumnTable = container.isColumnStore + if (isColumnTable && pr.getLocalMaxMemory > 0) { + val metaData = container.fetchHiveMetaData(false) + val schema = metaData.schema.asInstanceOf[StructType] + val numColumnsInTable = schema.length // Resetting PR numRows in cached batch as this will be calculated every time. var rowsInColumnBatch = 0L var offHeapSize = 0L if (container ne null) { - // TODO: this should use a transactional iterator to get a consistent - // snapshot (also pass the same transaction to getNumColumnsInTable - // for reading value and delete count) + // This deliberately collects uncommitted entries so that bulk insert + // progress can be monitored on dashboard. However, the pre-created transaction + // is used to check for committed entries in case there are multiple column + // batches that need to be merged val itr = new pr.PRLocalScanIterator(true /* primaryOnly */ , null /* no TX */ , null /* not required since includeValues is false */ , createRemoteIterator, false /* forUpdate */ , false /* includeValues */) + val maxDeltaRows = pr.getColumnMaxDeltaRows + val smallBatchBuckets = ObjectObjectHashMap.withExpectedSize[ + BucketRegion, mutable.ArrayBuffer[RegionEntry]](4) // using direct region operations while (itr.hasNext) { - val re = itr.next().asInstanceOf[AbstractRegionEntry] - val key = re.getRawKey.asInstanceOf[ColumnFormatKey] - if (numColumnsInTable < 0) { - numColumnsInTable = key.getNumColumnsInTable(table) - } - rowsInColumnBatch += key.getColumnBatchRowCount(itr, re, - numColumnsInTable) - re._getValue() match { - case v: ColumnFormatValue => offHeapSize += v.getOffHeapSizeInBytes - case _ => + val re = itr.next().asInstanceOf[RegionEntry] + if (!re.isDestroyedOrRemoved) { + val key = re.getRawKey.asInstanceOf[ColumnFormatKey] + val batchRowCount = key.getColumnBatchRowCount(itr, re, numColumnsInTable) + rowsInColumnBatch += batchRowCount + // check if bucket has multiple small batches + val br = itr.getHostedBucketRegion + if (key.getColumnIndex == ColumnFormatEntry.STATROW_COL_INDEX && + batchRowCount < maxDeltaRows) { + var batches = smallBatchBuckets.get(br) + if (batches eq null) { + batches = new mutable.ArrayBuffer[RegionEntry](2) + smallBatchBuckets.put(br, batches) + } + batches += re + } + re._getValue() match { + case v: ColumnFormatValue => offHeapSize += v.getOffHeapSizeInBytes + case _ => + } } } + // submit a task to merge small batches if required + if (smallBatchBuckets.size() > 0) { + mergeSmallColumnBatches(tx, pr, container, metaData, + smallBatchBuckets.asScala.filter(_._2.length > 1)) + } } val stats = pr.getPrStats stats.setPRNumRowsInColumnBatches(rowsInColumnBatch) stats.setOffHeapSizeInBytes(offHeapSize) + } else if (!isColumnTable && pr.getLocalMaxMemory > 0 && container.isRowBuffer) { + rolloverRowBuffers(pr) } } + } catch { + case t: Throwable => success = false; handleException(t) + } finally { + handleTransaction(cache, tx, context, success) + } + } + + private def minSizeForRollover(pr: PartitionedRegion): Int = + math.max(pr.getColumnMaxDeltaRows >>> 3, pr.getColumnMinDeltaRows) + + /** + * Check if row buffers are large and have not been touched for a while + * then roll it over into the column table + */ + private def rolloverRowBuffers(pr: PartitionedRegion): Unit = { + val localPrimaries = pr.getDataStore.getAllLocalPrimaryBucketRegions + if ((localPrimaries ne null) && localPrimaries.size() > 0) { + val doRollover = new Predicate[BucketRegion] { + private val minModTime = pr.getCache.cacheTimeMillis() - delayMillis + + override def test(br: BucketRegion): Boolean = { + br.getLastModifiedTime <= minModTime && br.getRegionSize >= minSizeForRollover(pr) + } + } + val rolloverBuckets = localPrimaries.asScala.filter( + br => doRollover.test(br) && !br.columnBatchFlushLock.isWriteLocked) + // enqueue a job to roll over required row buffers into column table + // (each bucket will perform a last minute check before rollover inside lock) + if (rolloverBuckets.nonEmpty) { + // logInfo( + // s"SW:111: will rollover buckets for ${pr.getFullPath}: ${rolloverBuckets.map(_.getId)}") + implicit val executionContext = Utils.executionContext(pr.getGemFireCache) + Future(rolloverBuckets.foreach(bucket => withExceptionHandling( + bucket.createAndInsertColumnBatch(null, true, doRollover)))) + } + } + } + + private val mergeTask = new AtomicReference[Future[Unit]]() + + /** + * Merge multiple column batches that are small in size in a bucket. + * These can get created due to a small "tail" in bulk imports (large enough + * to exceed minimal size that would have pushed them into row buffers), + * or a time-based flush that tolerates small sized column batches due to + * [[rolloverRowBuffers]] or a forced flush of even smaller size for sample tables. + */ + private def mergeSmallColumnBatches(tx: TXStateProxy, pr: PartitionedRegion, + container: GemFireContainer, metaData: ExternalTableMetaData, + smallBatches: mutable.Map[BucketRegion, mutable.ArrayBuffer[RegionEntry]]): Unit = { + if (mergeTask.get() ne null) return + // skip uncommitted entries for merge + val txState = tx.getTXStateForRead + // reverse iteration of entries so that remove does not change indices to be iterated + var skip = false + for ((br, entries) <- smallBatches; j <- (entries.length - 1) to 0) { + val entry = entries(j) + val re = entry match { + case e: AbstractRegionEntry => txState.getLocalEntry(pr, br, -1, e, false) + case _ => entry + } + if (re eq null) { + entries.remove(j) + if (entries.length <= 1) skip = true + } else if (re ne entry) { + entries(j) = re.asInstanceOf[RegionEntry] + } + } + // keep only batches with size > 1 + val batchBuckets = if (skip) smallBatches.filter(_._2.length > 1) else smallBatches + if (batchBuckets.nonEmpty) mergeTask.synchronized { + // synchronized instead of compareAndSet to avoid creating Future execution + if (mergeTask.get() ne null) return + logInfo( + s"Found small batches for ${pr.getName}: ${batchBuckets.map(_._2.map(_.getRawKey))}") + val cache = pr.getGemFireCache + implicit val executionContext = Utils.executionContext(cache) + mergeTask.set(Future(withExceptionHandling { + val tableName = container.getQualifiedTableName + val schema = metaData.schema.asInstanceOf[StructType] + val compileKey = tableName.concat(".MERGE_SMALL_BATCHES") + val gen = CodeGeneration.compileCode(compileKey, schema.fields, () => { + val schemaAttrs = Utils.schemaAttributes(schema) + val tableScan = ColumnTableScan(schemaAttrs, dataRDD = null, + otherRDDs = Nil, numBuckets = -1, partitionColumns = Nil, + partitionColumnAliases = Nil, baseRelation = null, schema, allFilters = Nil, + schemaAttrs, caseSensitive = true) + // reduce min delta row size to avoid going through rolloverRowBuffers again + val insertPlan = ColumnInsertExec(tableScan, Nil, Nil, + numBuckets = -1, isPartitioned = false, None, + (pr.getColumnBatchSize, minSizeForRollover(pr), metaData.compressionCodec), + tableName, onExecutor = true, schema, + metaData.externalStore.asInstanceOf[ExternalStore], useMemberVariables = false) + // now generate the code with the help of WholeStageCodegenExec + // this is only used for local code generation while its RDD semantics + // and related methods are all ignored + val (ctx, code) = ExternalStoreUtils.codeGenOnExecutor( + WholeStageCodegenExec(insertPlan), insertPlan) + val references = ctx.references + // also push the index of batchId reference at the end which can be + // used by caller to update the reference objects before execution + references += insertPlan.getBatchIdRef + (code, references.toArray) + }) + val references = gen._2.clone() + // full projection for the iterators + val numColumns = schema.length + val projection = (1 to numColumns).toArray + var success = false + var txState: TXStateProxy = null + var context: TXManagerImpl.TXContext = null + // for each bucket, create an iterator to scan and insert the result batches; + // a separate iterator is required because one ColumnInsertExec assumes a single batchId + for ((br, entries) <- batchBuckets) try { + success = false + // start a new transaction for each bucket + txState = null + txState = if (cache.snapshotEnabled) { + context = TXManagerImpl.getOrCreateTXContext() + cache.getCacheTransactionManager.beginTX(context, IsolationLevel.SNAPSHOT, null, null) + } else null + // update the bucketId as per the current bucket + val batchIdRef = references(references.length - 1).asInstanceOf[Int] + val bucketId = br.getId + references(batchIdRef + 1) = bucketId + val keys = entries.map(_.getRawKey.asInstanceOf[ColumnFormatKey]) + logInfo(s"Merging batches for ${pr.getName}:$bucketId :: $keys") + // no harm in passing a references array with an extra element at end + val iter = gen._1.generate(references).asInstanceOf[BufferedRowIterator] + // use the entries already determined for the iterator read by generated code + val batchIter = ColumnBatchIterator(br, entries.iterator, projection, context = null) + iter.init(bucketId, Array(Iterator[Any](new ResultSetTraversal(conn = null, stmt = null, + rs = null, context = null), batchIter).asInstanceOf[Iterator[InternalRow]])) + while (iter.hasNext) { + iter.next() // ignore result which is number of inserted rows + } + // now delete the keys that have been inserted above + logInfo(s"Deleting merged batches for ${pr.getName}:$bucketId :: $keys") + keys.foreach(ColumnDelta.deleteBatch(_, pr, numColumns)) + success = true + } catch { + case t: Throwable => handleException(t) + } finally { + handleTransaction(cache, txState, context, success) + } + })) } } } diff --git a/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala b/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala index 50f59e38d6..e91089187f 100644 --- a/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala +++ b/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala @@ -28,7 +28,6 @@ import scala.util.control.NonFatal import com.gemstone.gemfire.CancelException import com.pivotal.gemfirexd.Attribute import com.pivotal.gemfirexd.internal.engine.ui.{SnappyExternalTableStats, SnappyIndexStats, SnappyRegionStats} -import io.snappydata.Constant._ import org.apache.spark.SparkContext import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} @@ -62,9 +61,6 @@ object SnappyThinConnectorTableStatsProvider extends TableStatsProviderService { if (!doRun) { _url = url initializeConnection(Some(sc)) - // reduce default interval a bit - val delay = sc.getConf.getLong(Constant.SPARK_SNAPPY_PREFIX + - "calcTableSizeInterval", DEFAULT_CALC_TABLE_SIZE_SERVICE_INTERVAL) doRun = true new Timer("SnappyThinConnectorTableStatsProvider", true).schedule( new TimerTask { @@ -78,7 +74,7 @@ object SnappyThinConnectorTableStatsProvider extends TableStatsProviderService { case e: Exception => logError("SnappyThinConnectorTableStatsProvider", e) } } - }, delay, delay) + }, delayMillis, delayMillis) } } } diff --git a/core/src/main/scala/io/snappydata/TableStatsProviderService.scala b/core/src/main/scala/io/snappydata/TableStatsProviderService.scala index 94e62137e3..97bc76aa3a 100644 --- a/core/src/main/scala/io/snappydata/TableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/TableStatsProviderService.scala @@ -22,18 +22,19 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} import scala.language.implicitConversions import scala.util.control.NonFatal import com.gemstone.gemfire.CancelException +import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.ui.{SnappyExternalTableStats, SnappyIndexStats, SnappyRegionStats} +import io.snappydata.Constant.{DEFAULT_CALC_TABLE_SIZE_SERVICE_INTERVAL, PROPERTY_PREFIX, SPARK_SNAPPY_PREFIX} import org.apache.spark.sql.SnappySession import org.apache.spark.sql.collection.Utils -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.{Logging, SparkContext, SparkEnv} trait TableStatsProviderService extends Logging { @@ -45,6 +46,15 @@ trait TableStatsProviderService extends Logging { protected val membersInfo: mutable.Map[String, mutable.Map[String, Any]] = new ConcurrentHashMap[String, mutable.Map[String, Any]](8, 0.7f, 1).asScala + protected[snappydata] lazy val delayMillis: Long = SparkEnv.get match { + case null => DEFAULT_CALC_TABLE_SIZE_SERVICE_INTERVAL + case env => env.conf.getOption(PROPERTY_PREFIX + "calcTableSizeInterval") match { + case None => env.conf.getLong(SPARK_SNAPPY_PREFIX + "calcTableSizeInterval", + DEFAULT_CALC_TABLE_SIZE_SERVICE_INTERVAL) + case Some(v) => v.toLong + } + } + @GuardedBy("this") protected var memberStatsFuture: Option[Future[Unit]] = None protected val waitDuration = Duration(5000L, TimeUnit.MILLISECONDS) @@ -104,6 +114,7 @@ trait TableStatsProviderService extends Logging { val future = synchronized(memberStatsFuture match { case Some(f) => f case None => + implicit val executionContext = Utils.executionContext(Misc.getGemFireCacheNoThrow) val f = Future(fillAggregatedMemberStatsOnDemand()) memberStatsFuture = Some(f) f diff --git a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala index ae1382fc7b..3aa108fac5 100644 --- a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala +++ b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala @@ -22,7 +22,6 @@ import java.sql.SQLException import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} import scala.reflect.ClassTag @@ -33,6 +32,7 @@ import com.gemstone.gemfire.cache.LowMemoryException import com.gemstone.gemfire.internal.shared.ClientSharedUtils import com.gemstone.gemfire.internal.shared.unsafe.{DirectBufferAllocator, UnsafeHolder} import com.gemstone.gemfire.internal.{ByteArrayDataInput, ByteBufferDataOutput} +import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.shared.common.reference.SQLState import io.snappydata.Constant @@ -113,6 +113,7 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti private[sql] def startShuffleCleanups(sc: SparkContext): Unit = { val numShuffleDeps = shuffleDependencies.length + val cache = Misc.getGemFireCacheNoThrow if (numShuffleDeps > 0) { sc.cleaner match { case Some(cleaner) => @@ -120,6 +121,7 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti while (i < numShuffleDeps) { val shuffleDependency = shuffleDependencies(i) // Cleaning the shuffle artifacts asynchronously + implicit val executionContext = Utils.executionContext(cache) shuffleCleanups(i) = Future { cleaner.doCleanupShuffle(shuffleDependency, blocking = true) } diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index b559a160fb..657b5fb67c 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -23,6 +23,7 @@ import java.util.TimeZone import scala.annotation.tailrec import scala.collection.{mutable, Map => SMap} +import scala.concurrent.ExecutionContext import scala.language.existentials import scala.reflect.ClassTag import scala.util.Sorting @@ -30,6 +31,7 @@ import scala.util.control.NonFatal import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.gemstone.gemfire.internal.cache.GemFireCacheImpl import com.gemstone.gemfire.internal.shared.BufferAllocator import com.gemstone.gemfire.internal.shared.unsafe.UnsafeHolder import com.pivotal.gemfirexd.internal.engine.jdbc.GemFireXDRuntimeException @@ -45,7 +47,7 @@ import org.apache.spark.scheduler.TaskLocation import org.apache.spark.scheduler.local.LocalSchedulerBackend import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, GenericRow, UnsafeRow} import org.apache.spark.sql.catalyst.json.{JSONOptions, JacksonGenerator, JacksonUtils} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} @@ -389,6 +391,15 @@ object Utils { final def isLoner(sc: SparkContext): Boolean = (sc ne null) && sc.schedulerBackend.isInstanceOf[LocalSchedulerBackend] + def executionContext(cache: GemFireCacheImpl): ExecutionContext = { + if (cache eq null) scala.concurrent.ExecutionContext.Implicits.global + else { + val dm = cache.getDistributionManager + if (dm.isLoner) scala.concurrent.ExecutionContext.Implicits.global + else ExecutionContext.fromExecutorService(dm.getWaitingThreadPool) + } + } + def parseColumnsAsClob(s: String): (Boolean, Set[String]) = { if (s.trim.equals("*")) { (true, Set.empty[String]) @@ -514,6 +525,8 @@ object Utils { }: _*) } + def schemaAttributes(schema: StructType): Seq[AttributeReference] = schema.toAttributes + def getFields(o: Any): Map[String, Any] = { val fieldsAsPairs = for (field <- o.getClass.getDeclaredFields) yield { field.setAccessible(true) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala b/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala new file mode 100644 index 0000000000..4be32a06d3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala @@ -0,0 +1,67 @@ +/* + * Copyright (c) 2017 SnappyData, Inc. All rights reserved. + * + * Licensed 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. See accompanying + * LICENSE file. + */ +package org.apache.spark.sql.execution + +import java.text.NumberFormat +import java.util.Locale + +import scala.collection.mutable + +import com.gemstone.gnu.trove.TLongArrayList + +import org.apache.spark.SparkContext +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} + +/** Additional metric types supported by SnappyData. */ +object SnappyMetrics { + + /** + * Combine multiple "sum" metrics into a list of values for compact display. + * Metrics having the same "splitSumX" prefix will be added to the first + * accumulator in the series for display. For example, for metric types + * "splitSum0_0", "splitSum0_1", ... all will be collapsed into a list + * of values into "splitSum0_0" for display. + */ + val SPLIT_SUM_METRIC = "splitSum" + + /** + * Create a metric to report multiple sums as a single metric. All metrics are combined + * and displayed as comma-separated values against the name for "splitIndex" = 0. + */ + def createSplitSumMetric(sc: SparkContext, name: String, id: Int, splitIndex: Int): SQLMetric = { + // The final result of this metric in physical operator UI may looks like: + // data size total (min, med, max): + // 100GB (100MB, 1GB, 10GB) + val acc = new SQLMetric(SPLIT_SUM_METRIC + id + '_' + splitIndex) + acc.register(sc, name = Some(name), countFailedValues = false) + acc + } + + /** + * Aggregate the final accumulator results across tasks and display as a string + * for a single physical operator. + */ + def stringValue(metricType: String, values: Any): String = { + if (metricType.startsWith(SPLIT_SUM_METRIC)) { + val valueList = values.asInstanceOf[mutable.ArrayBuffer[TLongArrayList]] + val numberFormat = NumberFormat.getIntegerInstance(Locale.US) + valueList.collect { + case l if l ne null => numberFormat.format(l.toNativeArray.sum) + }.mkString(" / ") + } else SQLMetrics.stringValue(metricType, values.asInstanceOf[TLongArrayList].toNativeArray) + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala index ea1572a211..05f55a0fd2 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala @@ -172,3 +172,18 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor { childProduce } } + +/** + * An iterator that will update provided metrics (those supported by an implementation). + */ +trait IteratorWithMetrics[A] extends Iterator[A] { + + /** + * Set a metric to be updated during iteration. + * + * @param name name of the metric + * @param metric the metric to be updated during iteration + * @return true if given metric is supported else false + */ + def setMetric(name: String, metric: SQLMetric): Boolean +} diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala index 2aa5d04349..e17060ab57 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala @@ -36,6 +36,7 @@ import io.snappydata.thrift.common.BufferedBlob import org.apache.spark.memory.MemoryManagerCallback import org.apache.spark.sql.execution.columnar.encoding.{ColumnDecoder, ColumnDeleteDecoder, ColumnEncoding, UpdatedColumnDecoder, UpdatedColumnDecoderBase} import org.apache.spark.sql.execution.columnar.impl._ +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.row.PRValuesIterator import org.apache.spark.sql.store.CompressionUtils import org.apache.spark.sql.types.StructField @@ -110,7 +111,6 @@ abstract class ResultSetIterator[A](conn: Connection, case NonFatal(e) => logWarning("Exception closing statement", e) } hasNextValue = false - } } @@ -119,18 +119,26 @@ object ColumnBatchIterator { def apply(region: LocalRegion, bucketIds: java.util.Set[Integer], projection: Array[Int], fullScan: Boolean, context: TaskContext): ColumnBatchIterator = { - new ColumnBatchIterator(region, batch = null, bucketIds, projection, fullScan, context) + new ColumnBatchIterator(region, batch = null, statsEntries = null, bucketIds, + projection, fullScan, context) } def apply(batch: ColumnBatch): ColumnBatchIterator = { - new ColumnBatchIterator(region = null, batch, bucketIds = null, + new ColumnBatchIterator(region = null, batch, statsEntries = null, bucketIds = null, projection = null, fullScan = false, context = null) } + + def apply(bucketRegion: BucketRegion, statsEntries: Iterator[RegionEntry], + projection: Array[Int], context: TaskContext): ColumnBatchIterator = { + new ColumnBatchIterator(bucketRegion.getPartitionedRegion, batch = null, + statsEntries, java.util.Collections.singleton[Integer](bucketRegion.getId), + projection, fullScan = false, context) + } } -final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, - bucketIds: java.util.Set[Integer], projection: Array[Int], - fullScan: Boolean, context: TaskContext) +final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, + statsEntries: Iterator[RegionEntry], bucketIds: java.util.Set[Integer], + projection: Array[Int], fullScan: Boolean, context: TaskContext) extends PRValuesIterator[ByteBuffer](container = null, region, bucketIds) { if (region ne null) { @@ -144,11 +152,13 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, protected[sql] var currentVal: ByteBuffer = _ private var currentDeltaStats: ByteBuffer = _ - private var currentKeyPartitionId: Int = _ - private var currentKeyUUID: Long = _ + private var currentKey: ColumnFormatKey = _ private var batchProcessed = false private var currentColumns = new ArrayBuffer[ColumnFormatValue]() + private var diskBatchesFullMetric: SQLMetric = _ + private var diskBatchesPartialMetric: SQLMetric = _ + override protected def createIterator(container: GemFireContainer, region: LocalRegion, tx: TXStateInterface): PRIterator = if (region ne null) { val txState = if (tx ne null) tx.getLocalTXState else null @@ -156,7 +166,8 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, java.util.Iterator[RegionEntry]] { override def apply(br: BucketRegion, numEntries: java.lang.Long): java.util.Iterator[RegionEntry] = { - new ColumnFormatIterator(br, projection, fullScan, txState) + if (statsEntries eq null) new ColumnFormatIterator(br, projection, fullScan, txState) + else new ColumnFormatStatsIterator(br, statsEntries, tx) } } val createRemoteIterator = new BiFunction[java.lang.Integer, PRIterator, @@ -171,18 +182,17 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, false, true, true) } else null - def getCurrentBatchId: Long = currentKeyUUID + def getCurrentBatchId: Long = currentKey.uuid - def getCurrentBucketId: Int = currentKeyPartitionId + def getCurrentBucketId: Int = currentKey.partitionId private[execution] def getCurrentStatsColumn: ColumnFormatValue = currentColumns(0) private[sql] def getColumnBuffer(columnPosition: Int, throwIfMissing: Boolean): ByteBuffer = { val value = itr.getBucketEntriesIterator.asInstanceOf[ClusteredColumnIterator] - .getColumnValue(columnPosition) + .getColumnValue(columnPosition).asInstanceOf[ColumnFormatValue] if (value ne null) { - val columnValue = value.asInstanceOf[ColumnFormatValue].getValueRetain( - FetchRequest.DECOMPRESS) + val columnValue = value.getValueRetain(FetchRequest.DECOMPRESS) val buffer = columnValue.getBuffer if (buffer.remaining() > 0) { currentColumns += columnValue @@ -192,7 +202,7 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, if (throwIfMissing) { // empty buffer indicates value removed from region throw new EntryDestroyedException(s"Iteration on column=$columnPosition " + - s"partition=$currentKeyPartitionId batchUUID=$currentKeyUUID " + + s"partition=${currentKey.partitionId} batchUUID=${currentKey.uuid} " + "failed due to missing value") } else null } @@ -248,11 +258,22 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, if ((previousColumns ne null) && previousColumns.nonEmpty) { currentColumns = null val len = previousColumns.length + val checkDiskColumns = (diskBatchesFullMetric ne null) || (diskBatchesPartialMetric ne null) + var numDiskColumns = 0 var i = 0 while (i < len) { - previousColumns(i).release() + val v = previousColumns(i) + if (checkDiskColumns && (v.getRegionContext eq null)) numDiskColumns += 1 + v.release() i += 1 } + if (numDiskColumns > 0) { + if (numDiskColumns == len) { + if (diskBatchesFullMetric ne null) diskBatchesFullMetric.add(1) + } else if (diskBatchesPartialMetric ne null) { + diskBatchesPartialMetric.add(1) + } + } len } else 0 } @@ -280,8 +301,7 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, val buffer = columnValue.getBuffer // empty buffer indicates value removed from region if (buffer.remaining() > 0) { - currentKeyPartitionId = key.partitionId - currentKeyUUID = key.uuid + currentKey = key currentVal = buffer currentColumns += columnValue // check for update/delete stats row @@ -303,6 +323,12 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, } } + override def setMetric(name: String, metric: SQLMetric): Boolean = name match { + case ColumnTableScan.NUM_BATCHES_DISK_PARTIAL => diskBatchesPartialMetric = metric; true + case ColumnTableScan.NUM_BATCHES_DISK_FULL => diskBatchesFullMetric = metric; true + case _ => false + } + def close(): Unit = { if (itr ne null) { itr.close() diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala index 25687c386f..f04f69fc4f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatchCreator.scala @@ -96,7 +96,7 @@ final class ColumnBatchCreator( val references = ctx.references // also push the index of batchId reference at the end which can be // used by caller to update the reference objects before execution - references += insertPlan.batchIdRef + references += insertPlan.getBatchIdRef (code, references.toArray) }) val references = gen._2.clone() diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala index 360c3d44e0..170adcbfa0 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala @@ -75,7 +75,7 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String], @transient private var encoderArrayTerm: String = _ @transient private var cursorArrayTerm: String = _ - @transient private[sql] var batchIdRef = -1 + @transient private var batchIdRef: Int = -1 @transient private var batchBucketIdTerm: Option[String] = None @@ -89,6 +89,8 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String], override protected def isInsert: Boolean = true + def getBatchIdRef: Int = batchIdRef + /** Frequency of rows to check for total size exceeding batch size. */ private val (checkFrequency, checkMask) = { val batchSize = columnBatchSize diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala index b7360bdaea..929fe9baec 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala @@ -48,6 +48,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer} import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.columnar.ColumnTableScan.{NUM_BATCHES_DISK_FULL, NUM_BATCHES_DISK_PARTIAL, NUM_ROWS_DISK} import org.apache.spark.sql.execution.columnar.encoding._ import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, ColumnDelta} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} @@ -64,7 +65,7 @@ import org.apache.spark.{Dependency, Logging, Partition, RangeDependency, SparkC * This plan overrides outputPartitioning and makes it inline with the * partitioning of the underlying DataSource. */ -private[sql] final case class ColumnTableScan( +final case class ColumnTableScan( output: Seq[Attribute], dataRDD: RDD[Any], otherRDDs: Seq[RDD[InternalRow]], @@ -102,14 +103,20 @@ private[sql] final case class ColumnTableScan( override def getMetrics: Map[String, SQLMetric] = { if (sqlContext eq null) Map.empty else super.getMetrics ++ Map( - "numRowsBuffer" -> SQLMetrics.createMetric(sparkContext, - "number of output rows from row buffer"), - "columnBatchesSeen" -> SQLMetrics.createMetric(sparkContext, - "column batches seen"), - "updatedColumnCount" -> SQLMetrics.createMetric(sparkContext, - "total updated columns in batches"), - "deletedBatchCount" -> SQLMetrics.createMetric(sparkContext, - "column batches having deletes"), + "numRowsBuffer" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "rows read from row buffer (total / disk)", 0, 0), + NUM_ROWS_DISK -> SnappyMetrics.createSplitSumMetric(sparkContext, + "number of rows on disk from row buffer", 0, 1), + "columnBatchesSeen" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "column batches read (total / disk-partial / disk-full)", 1, 0), + NUM_BATCHES_DISK_PARTIAL -> SnappyMetrics.createSplitSumMetric(sparkContext, + "column batches read from disk (partial)", 1, 1), + NUM_BATCHES_DISK_FULL -> SnappyMetrics.createSplitSumMetric(sparkContext, + "column batches read from disk (full)", 1, 2), + "updatedColumnCount" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "deltas (updated columns / batches having deletes)", 2, 0), + "deletedBatchCount" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "column batches having deletes", 2, 1), "columnBatchesSkipped" -> SQLMetrics.createMetric(sparkContext, "column batches skipped by the predicate")) ++ ( if (otherRDDs.isEmpty) Map.empty @@ -177,10 +184,13 @@ private[sql] final case class ColumnTableScan( override def doProduce(ctx: CodegenContext): String = { val numOutputRows = metricTerm(ctx, "numOutputRows") val numRowsBuffer = metricTerm(ctx, "numRowsBuffer") + val numRowsBufferDisk = metricTerm(ctx, NUM_ROWS_DISK) + val numBatchesDiskPartial = metricTerm(ctx, NUM_BATCHES_DISK_PARTIAL) + val numBatchesDiskFull = metricTerm(ctx, NUM_BATCHES_DISK_FULL) val numRowsOther = if (otherRDDs.isEmpty) null else metricTerm(ctx, "numRowsOtherRDDs") val embedded = (baseRelation eq null) || - (baseRelation.connectionType == ConnectionType.Embedded) + (baseRelation.connectionType == ConnectionType.Embedded) // PartitionedPhysicalRDD always has one input. // It returns an iterator of iterators (row + column) // except when doing union with multiple RDDs where other @@ -216,6 +226,7 @@ private[sql] final case class ColumnTableScan( val iteratorClass = "scala.collection.Iterator" val colIteratorClass = if (embedded) classOf[ColumnBatchIterator].getName else classOf[ColumnBatchIteratorOnRS].getName + val iteratorWithMetricsClass = classOf[IteratorWithMetrics[_]].getName if (otherRDDs.isEmpty) { if (isForSampleReservoirAsRegion) { ctx.addMutableState(iteratorClass, rowInputSRR, @@ -323,6 +334,24 @@ private[sql] final case class ColumnTableScan( } """ } + val setRowDiskMetricsSnippet = if (numRowsBufferDisk eq null) "" + else { + s""" + if ($rowInput instanceof $iteratorWithMetricsClass) { + (($iteratorWithMetricsClass)$rowInput).setMetric("$NUM_ROWS_DISK", $numRowsBufferDisk); + } + """ + } + val setColumnDiskMetricsSnippet = if (numBatchesDiskPartial eq null) "" + else { + s""" + if ($colInput instanceof $iteratorWithMetricsClass) { + $iteratorWithMetricsClass mIter = ($iteratorWithMetricsClass)$colInput; + mIter.setMetric("$NUM_BATCHES_DISK_PARTIAL", $numBatchesDiskPartial); + mIter.setMetric("$NUM_BATCHES_DISK_FULL", $numBatchesDiskFull); + } + """ + } val initRowTableDecoders = new StringBuilder val bufferInitCodeBlocks = new ArrayBuffer[String]() @@ -476,7 +505,8 @@ private[sql] final case class ColumnTableScan( val deltaStatsRow = ctx.freshName("deltaStatsRow") val colNextBytes = ctx.freshName("colNextBytes") val numTableColumns = if (ordinalIdTerm eq null) relationSchema.size - else relationSchema.size - ColumnDelta.mutableKeyNames.length // for update/delete + // for update/delete + else relationSchema.size - ColumnDelta.mutableKeyNames.length val numColumnsInStatBlob = ColumnStatsSchema.numStatsColumns(numTableColumns) val incrementBatchOutputRows = if (numOutputRows ne null) { @@ -616,6 +646,7 @@ private[sql] final case class ColumnTableScan( |// case when partition is of otherRDDs by iterating over it |// using an UnsafeRow adapter. |try { + | $setRowDiskMetricsSnippet$setColumnDiskMetricsSnippet | while ($nextBatch()) { | $bufferInitCodeStr | $assignBatchId @@ -794,6 +825,10 @@ private[sql] final case class ColumnTableScan( object ColumnTableScan extends Logging { + val NUM_ROWS_DISK = "numRowsBufferDisk" + val NUM_BATCHES_DISK_PARTIAL = "columnBatchesDiskPartial" + val NUM_BATCHES_DISK_FULL = "columnBatchesDiskFull" + def generateStatPredicate(ctx: CodegenContext, isColumnTable: Boolean, schemaAttrs: Seq[AttributeReference], allFilters: Seq[Expression], numRowsTerm: String, metricTerm: (CodegenContext, String) => String, metricAdd: String => String): String = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala index a24bd4c769..6e63bb87ad 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala @@ -296,8 +296,7 @@ object ColumnDelta { * Delete entire batch from column store for the batchId and partitionId * matching those of given key. */ - private[columnar] def deleteBatch(key: ColumnFormatKey, columnRegion: Region[_, _], - columnTableName: String): Unit = { + def deleteBatch(key: ColumnFormatKey, columnRegion: Region[_, _], numColumns: Int): Unit = { // delete all the rows with matching batchId def destroyKey(key: ColumnFormatKey): Unit = { @@ -308,7 +307,6 @@ object ColumnDelta { } } - val numColumns = key.getNumColumnsInTable(columnTableName) // delete the stats rows first destroyKey(key.withColumnIndex(ColumnFormatEntry.STATROW_COL_INDEX)) destroyKey(key.withColumnIndex(ColumnFormatEntry.DELTA_STATROW_COL_INDEX)) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala index 129bac8d81..d8ed2a8df2 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala @@ -32,6 +32,7 @@ import io.snappydata.thrift.internal.ClientBlob import org.apache.spark.sql.execution.columnar.encoding.ColumnDeleteDelta import org.apache.spark.sql.store.CompressionCodecId +import org.apache.spark.sql.types.StructType /** * A [[RowEncoder]] implementation for [[ColumnFormatValue]] and child classes. @@ -134,8 +135,9 @@ final class ColumnFormatEncoder extends RowEncoder { deleteDelta.release() } if (deleteBatch) { - ColumnDelta.deleteBatch(deleteKey, region, - region.getUserAttribute.asInstanceOf[GemFireContainer].getQualifiedTableName) + val container = region.getUserAttribute.asInstanceOf[GemFireContainer] + val schema = container.fetchHiveMetaData(false).schema.asInstanceOf[StructType] + ColumnDelta.deleteBatch(deleteKey, region, schema.length) } case _ => }) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala index de241e813c..ad5393d738 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala @@ -33,8 +33,7 @@ import com.gemstone.gemfire.internal.shared._ import com.gemstone.gemfire.internal.shared.unsafe.DirectBufferAllocator import com.gemstone.gemfire.internal.size.ReflectionSingleObjectSizer.REFERENCE_SIZE import com.gemstone.gemfire.internal.{ByteBufferDataInput, DSCODE, DSFIDFactory, DataSerializableFixedID, HeapDataOutputStream} -import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils -import com.pivotal.gemfirexd.internal.engine.store.RegionKey +import com.pivotal.gemfirexd.internal.engine.store.{GemFireContainer, RegionKey} import com.pivotal.gemfirexd.internal.engine.{GfxdDataSerializable, GfxdSerializable, Misc} import com.pivotal.gemfirexd.internal.iapi.types.{DataValueDescriptor, SQLInteger, SQLLongint} import com.pivotal.gemfirexd.internal.impl.sql.compile.TableName @@ -46,6 +45,7 @@ import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.encoding.{ColumnDeleteDelta, ColumnEncoding, ColumnStatsSchema} import org.apache.spark.sql.execution.columnar.impl.ColumnFormatEntry.alignedSize import org.apache.spark.sql.store.{CompressionCodecId, CompressionUtils} +import org.apache.spark.sql.types.StructType /** * Utility methods for column format storage keys and values. @@ -106,18 +106,15 @@ final class ColumnFormatKey(private[columnar] var uuid: Long, // to be used only by deserialization def this() = this(-1L, -1, -1) - override def getNumColumnsInTable(columnTableName: String): Int = { - val bufferTable = ColumnFormatRelation.getTableName(columnTableName) - GemFireXDUtils.getGemFireContainer(bufferTable, true).getNumColumns - 1 - } + override def getNumColumnsInTable(columnTable: GemFireContainer): Int = + columnTable.fetchHiveMetaData(false).schema.asInstanceOf[StructType].length override def getColumnBatchRowCount(itr: PREntriesIterator[_], - re: AbstractRegionEntry, numColumnsInTable: Int): Int = { + re: RegionEntry, numColumnsInTable: Int): Int = { val currentBucketRegion = itr.getHostedBucketRegion - if ((columnIndex == ColumnFormatEntry.STATROW_COL_INDEX || + if (columnIndex == ColumnFormatEntry.STATROW_COL_INDEX || columnIndex == ColumnFormatEntry.DELTA_STATROW_COL_INDEX || - columnIndex == ColumnFormatEntry.DELETE_MASK_COL_INDEX) && - !re.isDestroyedOrRemoved) { + columnIndex == ColumnFormatEntry.DELETE_MASK_COL_INDEX) { val statsOrDeleteVal = re.getValue(currentBucketRegion) if (statsOrDeleteVal ne null) { val statsOrDelete = statsOrDeleteVal.asInstanceOf[ColumnFormatValue] @@ -320,7 +317,7 @@ class ColumnFormatValue extends SerializedDiskBuffer } @GuardedBy("this") - protected final def isCompressed: Boolean = decompressionState == 0 + final def isCompressed: Boolean = decompressionState == 0 override final def copyToHeap(owner: String): Unit = synchronized { columnBuffer = HeapBufferAllocator.instance().transfer(columnBuffer, owner) @@ -598,8 +595,9 @@ class ColumnFormatValue extends SerializedDiskBuffer override final def setDiskEntry(entry: AbstractOplogDiskRegionEntry, context: RegionEntryContext): Unit = synchronized { this.entry = entry - // set/update diskRegion only if incoming value has been provided - if (context ne null) { + // set/update diskRegion only if incoming value has been provided and is "better" + if ((context ne null) && + (!this.regionContext.isInstanceOf[LocalRegion] || context.isInstanceOf[LocalRegion])) { this.regionContext = context val codec = context.getColumnCompressionCodec if (codec ne null) { @@ -608,6 +606,8 @@ class ColumnFormatValue extends SerializedDiskBuffer } } + override def getRegionContext: RegionEntryContext = this.regionContext + override final def write(channel: OutputStreamChannel): Unit = { // write the pre-serialized buffer as is // Oplog layer will get compressed form by calling getValueRetain diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala index aaf7229b1e..f8ac46a388 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala @@ -411,3 +411,40 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, } } } + +/** + * A customized iterator for a single bucket of a column table that uses a list of stats rows + * to fetch entire batches as required by ColumnTableScan. This does not honour + * disk order so should be used only for a small list of rows while other cases + * should use the normal ColumnFormatIterator. + */ +final class ColumnFormatStatsIterator(bucketRegion: BucketRegion, + statsEntries: Iterator[RegionEntry], tx: TXStateInterface) extends ClusteredColumnIterator { + + try { + bucketRegion.checkReadiness() + } catch { + case e: RegionDestroyedException => if (bucketRegion.isUsedForPartitionedRegionBucket) { + bucketRegion.getPartitionedRegion.checkReadiness() + throw new BucketNotFoundException(e.getMessage) + } else throw e + } + + private var currentKey: ColumnFormatKey = _ + + override def hasNext: Boolean = statsEntries.hasNext + + override def next(): RegionEntry = { + val re = statsEntries.next() + currentKey = re.getRawKey.asInstanceOf[ColumnFormatKey] + assert(currentKey.getColumnIndex == ColumnFormatEntry.STATROW_COL_INDEX) + re + } + + override def getColumnValue(columnIndex: Int): AnyRef = { + val key = currentKey.withColumnIndex(columnIndex) + bucketRegion.get(key, null, false, true, false, null, tx, null, null, false, false) + } + + override def close(): Unit = currentKey = null +} diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index 29d726bece..ef3c9d6343 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -242,7 +242,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie // check for full batch delete if (ColumnDelta.checkBatchDeleted(buffer)) { - ColumnDelta.deleteBatch(key, region, columnTableName) + ColumnDelta.deleteBatch(key, region, schema.length) return } region.put(key, value) @@ -345,7 +345,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie * during iteration. We are not cleaning up the partial inserts of cached * batches for now. */ - private def doSnappyInsertOrPut(region: LocalRegion, batch: ColumnBatch, + private def doSnappyInsertOrPut(region: PartitionedRegion, batch: ColumnBatch, batchId: Long, partitionId: Int, maxDeltaRows: Int, compressionCodecId: Int): Unit = { val deltaUpdate = batch.deltaIndexes ne null val statRowIndex = if (deltaUpdate) ColumnFormatEntry.DELTA_STATROW_COL_INDEX @@ -366,6 +366,12 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie } // add the stats row val key = new ColumnFormatKey(batchId, partitionId, statRowIndex) + if (maxDeltaRows > 0 && maxDeltaRows < region.getColumnMaxDeltaRows) { + // log at info level for the case of column batch merges + logInfo(s"Putting batch of size = ${batch.numRows} into ${region.getName}: $key") + } else { + logDebug(s"Putting batch of size = ${batch.numRows} into ${region.getName}: $key") + } val allocator = Misc.getGemFireCache.getBufferAllocator val statsBuffer = Utils.createStatsBuffer(batch.statsData, allocator) val value = if (deltaUpdate) { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index 04d127fbec..911467f970 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -42,9 +42,10 @@ import org.apache.spark.serializer.ConnectionPropertiesSerializer import org.apache.spark.sql.SnappySession import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.collection.MultiBucketExecutorPartition -import org.apache.spark.sql.execution.columnar.{ExternalStoreUtils, ResultSetIterator} +import org.apache.spark.sql.execution.columnar.{ColumnTableScan, ExternalStoreUtils, ResultSetIterator} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.sources.StoreDataSourceStrategy.translateToFilter -import org.apache.spark.sql.execution.{RDDKryo, SecurityUtils} +import org.apache.spark.sql.execution.{IteratorWithMetrics, RDDKryo, SecurityUtils} import org.apache.spark.sql.sources.JdbcExtendedUtils.quotedName import org.apache.spark.sql.sources._ import org.apache.spark.{Partition, TaskContext} @@ -295,7 +296,9 @@ class RowFormatScanRDD(@transient val session: SnappySession, } val txId = if (tx ne null) tx.getTransactionId else null - val itr = new CompactExecRowIteratorOnScan(container, bucketIds, txId) + // always fault-in for row buffers + val itr = new CompactExecRowIteratorOnScan(container, bucketIds, txId, + faultIn = container.isRowBuffer) if (useResultSet) { // row buffer of column table: wrap a result set around the scan val dataItr = itr.map(r => @@ -436,7 +439,7 @@ final class CompactExecRowIteratorOnRS(conn: Connection, } abstract class PRValuesIterator[T](container: GemFireContainer, - region: LocalRegion, bucketIds: java.util.Set[Integer]) extends Iterator[T] { + region: LocalRegion, bucketIds: java.util.Set[Integer]) extends IteratorWithMetrics[T] { protected type PRIterator = PartitionedRegion#PRLocalScanIterator @@ -448,12 +451,10 @@ abstract class PRValuesIterator[T](container: GemFireContainer, protected def createIterator(container: GemFireContainer, region: LocalRegion, tx: TXStateInterface): PRIterator = if (container ne null) { - container.getEntrySetIteratorForBucketSet( - bucketIds.asInstanceOf[java.util.Set[Integer]], null, tx, 0, + container.getEntrySetIteratorForBucketSet(bucketIds, null, tx, 0, false, true).asInstanceOf[PRIterator] } else if (region ne null) { - region.getDataView(tx).getLocalEntriesIterator( - bucketIds.asInstanceOf[java.util.Set[Integer]], false, false, true, + region.getDataView(tx).getLocalEntriesIterator(bucketIds, false, false, true, region, true).asInstanceOf[PRIterator] } else null @@ -479,24 +480,40 @@ abstract class PRValuesIterator[T](container: GemFireContainer, } final class CompactExecRowIteratorOnScan(container: GemFireContainer, - bucketIds: java.util.Set[Integer], txId: TXId) + bucketIds: java.util.Set[Integer], txId: TXId, faultIn: Boolean) extends PRValuesIterator[AbstractCompactExecRow](container, region = null, bucketIds) { override protected[sql] val currentVal: AbstractCompactExecRow = container .newTemplateRow().asInstanceOf[AbstractCompactExecRow] + private var diskRowsMetric: SQLMetric = _ override protected[sql] def moveNext(): Unit = { val itr = this.itr while (itr.hasNext) { - val rl = itr.next() + val rl = itr.next().asInstanceOf[RowLocation] val owner = itr.getHostedBucketRegion - if (((owner ne null) || rl.isInstanceOf[NonLocalRegionEntry]) && - RegionEntryUtils.fillRowWithoutFaultInOptimized(container, owner, - rl.asInstanceOf[RowLocation], currentVal)) { - return + if ((owner ne null) || rl.isInstanceOf[NonLocalRegionEntry]) { + val valueWasNull = (diskRowsMetric ne null) && rl.isValueNull + if (faultIn) { + if (RegionEntryUtils.fillRowFaultInOptimized(container, owner, rl, currentVal)) { + if (valueWasNull) diskRowsMetric.add(1) + return + } + } else if (RegionEntryUtils.fillRowWithoutFaultInOptimized( + container, owner, rl, currentVal)) { + if (valueWasNull) diskRowsMetric.add(1) + return + } } } hasNextValue = false } + + override def setMetric(name: String, metric: SQLMetric): Boolean = { + if (name == ColumnTableScan.NUM_ROWS_DISK) { + diskRowsMetric = metric + true + } else false + } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala index 057576eff1..bb345e8758 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala @@ -16,11 +16,18 @@ */ package org.apache.spark.sql.execution.ui +import java.util.AbstractMap.SimpleEntry +import java.util.function.BiFunction + +import scala.collection.JavaConverters._ import scala.collection.mutable +import com.gemstone.gnu.trove.TLongArrayList +import io.snappydata.collection.ObjectObjectHashMap + import org.apache.spark.scheduler.{SparkListenerEvent, SparkListenerJobStart} import org.apache.spark.sql.CachedDataFrame -import org.apache.spark.sql.execution.{SQLExecution, SparkPlanInfo} +import org.apache.spark.sql.execution.{SQLExecution, SnappyMetrics, SparkPlanInfo} import org.apache.spark.{JobExecutionStatus, SparkConf} /** @@ -150,6 +157,92 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { } case _ => super.onOtherEvent(event) } + } + + /** + * Get all accumulator updates from all tasks which belong to this execution and merge them. + */ + override def getExecutionMetrics(executionId: Long): Map[Long, String] = synchronized { + baseExecutionIdToData.get(executionId) match { + case Some(executionUIData) => + val accumulatorUpdates = { + for (stageId <- executionUIData.stages; + stageMetrics <- baseStageIdToStageMetrics.get(stageId).toIterable; + taskMetrics <- stageMetrics.taskIdToMetricUpdates.values; + accumulatorUpdate <- taskMetrics.accumulatorUpdates) yield { + (accumulatorUpdate._1, accumulatorUpdate._2) + } + } + + val driverUpdates = executionUIData.driverAccumUpdates.toSeq + val totalUpdates = (accumulatorUpdates ++ driverUpdates).filter { + case (id, _) => executionUIData.accumulatorMetrics.contains(id) + } + mergeAccumulatorUpdates(totalUpdates, accumulatorId => + executionUIData.accumulatorMetrics(accumulatorId).metricType) + case None => + // This execution has been dropped + Map.empty + } + } + + private def mergeAccumulatorUpdates( + accumulatorUpdates: Seq[(Long, Any)], + metricTypeFunc: Long => String): Map[Long, String] = { + // Group by accumulatorId but also group on splitSum metric + // to include display of all into the first accumulator of a split series. + // The map below either has accumulatorId as key or the splitSum metric series type + // as the key, and second part of value is metric type for former case while + // accumulatorId of the first in series for latter case. + type MapValue = SimpleEntry[Any, Any] + val accumulatorMap = ObjectObjectHashMap.withExpectedSize[Any, MapValue](8) + for ((accumulatorId, value) <- accumulatorUpdates) { + val metricType = metricTypeFunc(accumulatorId) + if (metricType.startsWith(SnappyMetrics.SPLIT_SUM_METRIC)) { + val splitIndex = metricType.indexOf('_') + val key = metricType.substring(0, splitIndex) + val index = metricType.substring(splitIndex + 1).toInt + accumulatorMap.compute(key, new BiFunction[Any, MapValue, MapValue] { + private def expandBuffer(b: mutable.ArrayBuffer[TLongArrayList], size: Int): Unit = { + var i = b.length + while (i < size) { + b += null + i += 1 + } + } + override def apply(key: Any, v: MapValue): MapValue = { + val mapValue = if (v ne null) v + else new MapValue(new mutable.ArrayBuffer[TLongArrayList](math.max(index + 1, 4)), 0L) + val valueList = mapValue.getKey.asInstanceOf[mutable.ArrayBuffer[TLongArrayList]] + expandBuffer(valueList, index + 1) + val values = valueList(index) match { + case null => val l = new TLongArrayList(4); valueList(index) = l; l + case l => l + } + values.add(value.asInstanceOf[Long]) + if (index == 0) mapValue.setValue(accumulatorId) + mapValue + } + }) + } else { + accumulatorMap.compute(accumulatorId, new BiFunction[Any, MapValue, MapValue] { + override def apply(key: Any, v: MapValue): MapValue = { + val mapValue = if (v ne null) v + else new MapValue(new TLongArrayList(4), metricType) + mapValue.getKey.asInstanceOf[TLongArrayList].add(value.asInstanceOf[Long]) + mapValue + } + }) + } + } + // now create a map on accumulatorId and the values (which are either a + // list of longs or a list of list of longs) as string + accumulatorMap.asScala.map { + case (id: Long, entry) => + id -> SnappyMetrics.stringValue(entry.getValue.asInstanceOf[String], entry.getKey) + case (metricType: String, entry) => + entry.getValue.asInstanceOf[Long] -> SnappyMetrics.stringValue(metricType, entry.getKey) + }.toMap } } diff --git a/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala b/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala index 3fa2a311fb..8479b1f98f 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala @@ -40,10 +40,10 @@ object CompressionCodecId extends Enumeration { /** * The case of codec > MAX_ID should ideally be error but due to backward compatibility * the stats row does not have any header to determine compression or not so can fail - * in rare cases if first integer is a negative value. However it should never be match - * with the IDs here because negative of codecId which is written are -1, -2, -3 resolve - * to 0xfffffff... which should never happen since nullCount fields are non-nullable - * in the UnsafeRow created, so bitset cannot have 'ff' kind of patterns. + * in rare cases if first integer is a negative value. However it should never match + * with the IDs here because negative of codecIds are -1, -2, -3 that resolve to + * 0xfffffff... which should never happen since the first numRows field is non-nullable + * in the UnsafeRow created, so bitset cannot start with 'f'. */ def isCompressed(codec: Int): Boolean = codec > 0 && codec <= MAX_ID diff --git a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala index 48d9e75f19..ae1992b194 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala @@ -419,7 +419,8 @@ object StoreUtils { parameters.remove(PARTITIONER).foreach(v => sb.append(GEM_PARTITIONER).append('\'').append(v).append("' ")) - val overflow = parameters.get(OVERFLOW).forall(_.toBoolean) + // no overflow for row buffer tables + val overflow = (isRowTable || isShadowTable) && parameters.get(OVERFLOW).forall(_.toBoolean) val defaultEviction = if (overflow) s"$GEM_HEAPPERCENT $GEM_OVERFLOW" else EMPTY_STRING sb.append(parameters.remove(EVICTION_BY).map(v => { if (v.contains(LRUCOUNT) && isShadowTable) { @@ -459,7 +460,8 @@ object StoreUtils { parameters.remove(DISKSTORE) match { case Some(v) => if (!isPersistent && !overflow) { - throw Utils.analysisException(s"Option '$DISKSTORE' requires '$PERSISTENCE' option") + throw Utils.analysisException( + s"Option '$DISKSTORE' requires '$PERSISTENCE' or '$OVERFLOW' option") } if (v == GfxdConstants.GFXD_DEFAULT_DISKSTORE_NAME) { sb.append(s"'${GfxdConstants.SNAPPY_DEFAULT_DELTA_DISKSTORE}' ") @@ -473,7 +475,7 @@ object StoreUtils { parameters.remove(DISKSTORE).foreach { v => if (isPersistent) sb.append(s"'$v' ") else if (!isPersistent && !overflow) throw Utils.analysisException( - s"Option '$DISKSTORE' requires '$PERSISTENCE' option") + s"Option '$DISKSTORE' requires '$PERSISTENCE' or '$OVERFLOW' option") } } sb.append(parameters.remove(SERVER_GROUPS) diff --git a/store b/store index 7b6450d26c..e76e762c2a 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 7b6450d26c514d4f556a4092d778444a6177669d +Subproject commit e76e762c2a0371b6db42058725ec7533e95664a1 From cd21c4d15dded7c5ce6eee11f266b8898072ab7f Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 31 May 2018 23:31:15 +0530 Subject: [PATCH 04/19] update store link --- store | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/store b/store index e76e762c2a..4838032a21 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit e76e762c2a0371b6db42058725ec7533e95664a1 +Subproject commit 4838032a21daf03cad933cac8fb728426c99c759 From 00f6cda2b9bcbbfe5cf649aebfdd7dda08a2a826 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 5 Jun 2018 03:07:18 +0530 Subject: [PATCH 05/19] Some optimizations and fixed few issues - fixed disk metrics collection added previously; set the metric correctly for both row buffer iterator (ResultSetTraversal) and ColumnFormatIterator - added a metric for remote batch fetch - fixed multiple ColumnTableScans causing split metrics to add up into one ColumnTableScan; now use a unique ID for split metrics for each ColumnTableScan instance - fix an NPE in SnappyTableStatsProviderService while filling up result map from members since CHM cannot hold null values - use a common entry map in ColumnFormatIterator disk iteration instead of creating separate for every column batch - added implementation of PURGE_CODEGEN_CACHES as StoreCallbacksImpl.clearCodegenCaches - limit to one task per table for background rolloverRowBuffer and mergeSmallBatches tasks - replaced a few usage of Map.put with justPut for koloboke maps --- .../SnappyTableStatsProviderService.scala | 310 +++++++++--------- .../impl/SmartConnectorRDDHelper.scala | 4 +- .../apache/spark/sql/collection/Utils.scala | 2 +- .../spark/sql/execution/SnappyMetrics.scala | 23 +- .../sql/execution/columnar/ColumnBatch.scala | 22 +- .../execution/columnar/ColumnTableScan.scala | 55 ++-- .../impl/ClusteredColumnIterator.scala | 22 +- .../columnar/impl/ColumnFormatIterator.scala | 82 ++--- .../columnar/impl/RemoteEntriesIterator.scala | 9 + .../columnar/impl/StoreCallbacksImpl.scala | 9 + .../sql/execution/row/RowFormatScanRDD.scala | 23 +- .../sql/execution/ui/SnappySQLListener.scala | 53 ++- .../sql/hive/SnappyStoreHiveCatalog.scala | 7 +- .../apache/spark/sql/store/StoreUtils.scala | 15 +- store | 2 +- 15 files changed, 355 insertions(+), 283 deletions(-) diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index 44af11a074..4e77b04232 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -19,9 +19,9 @@ package io.snappydata -import java.util.concurrent.atomic.AtomicReference +import java.util.Collections import java.util.concurrent.{ConcurrentHashMap, TimeUnit} -import java.util.function.{BiFunction, Predicate} +import java.util.function.{BiFunction, Predicate, Function => JFunction} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -145,19 +145,21 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val o = itr.next().asInstanceOf[ListResultCollectorValue] val memMap = o.resultOfSingleExecution.asInstanceOf[java.util.HashMap[String, Any]] val map = new ConcurrentHashMap[String, Any](8, 0.7f, 1).asScala - val keyItr = memMap.keySet().iterator() + val entryItr = memMap.entrySet().iterator() - while (keyItr.hasNext) { - val key = keyItr.next() - map.put(key, memMap.get(key)) + while (entryItr.hasNext) { + val e = entryItr.next() + if ((e.getKey ne null) && e.getValue != null) { + map.put(e.getKey, e.getValue) + } } map.put("status", "Running") val dssUUID = memMap.get("diskStoreUUID").asInstanceOf[java.util.UUID] if (dssUUID != null) { - members.put(dssUUID.toString, map) + members.justPut(dssUUID.toString, map) } else { - members.put(memMap.get("id").asInstanceOf[String], map) + members.justPut(memMap.get("id").asInstanceOf[String], map) } } membersInfo ++= members.asScala @@ -257,11 +259,13 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService } } - private def withExceptionHandling(f: => Unit): Unit = { + private def withExceptionHandling(f: => Unit, doFinally: () => Unit = null): Unit = { try { f } catch { case t: Throwable => handleException(t) + } finally { + if (doFinally ne null) doFinally() } } @@ -270,15 +274,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val cache = Misc.getGemFireCache val regions = asSerializable(cache.getApplicationRegions.asScala) - // Transaction started to check for committed entries if required. - val txManager = cache.getCacheTransactionManager - var context: TXManagerImpl.TXContext = null - val tx = if (cache.snapshotEnabled) { - context = TXManagerImpl.getOrCreateTXContext() - txManager.beginTX(context, IsolationLevel.SNAPSHOT, null, null) - } else null - var success = true - try for (region: LocalRegion <- regions) { + for (region: LocalRegion <- regions) { if (region.getDataPolicy.withPartitioning()) { val pr = region.asInstanceOf[PartitionedRegion] val container = pr.getUserAttribute.asInstanceOf[GemFireContainer] @@ -299,8 +295,8 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService null /* not required since includeValues is false */ , createRemoteIterator, false /* forUpdate */ , false /* includeValues */) val maxDeltaRows = pr.getColumnMaxDeltaRows - val smallBatchBuckets = ObjectObjectHashMap.withExpectedSize[ - BucketRegion, mutable.ArrayBuffer[RegionEntry]](4) + var smallBucketRegion: BucketRegion = null + val smallBatchBuckets = new mutable.ArrayBuffer[BucketRegion](2) // using direct region operations while (itr.hasNext) { val re = itr.next().asInstanceOf[RegionEntry] @@ -309,15 +305,11 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val batchRowCount = key.getColumnBatchRowCount(itr, re, numColumnsInTable) rowsInColumnBatch += batchRowCount // check if bucket has multiple small batches - val br = itr.getHostedBucketRegion if (key.getColumnIndex == ColumnFormatEntry.STATROW_COL_INDEX && batchRowCount < maxDeltaRows) { - var batches = smallBatchBuckets.get(br) - if (batches eq null) { - batches = new mutable.ArrayBuffer[RegionEntry](2) - smallBatchBuckets.put(br, batches) - } - batches += re + val br = itr.getHostedBucketRegion + if (br eq smallBucketRegion) smallBatchBuckets += br + else smallBucketRegion = br } re._getValue() match { case v: ColumnFormatValue => offHeapSize += v.getOffHeapSizeInBytes @@ -325,26 +317,26 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService } } } + itr.close() // submit a task to merge small batches if required - if (smallBatchBuckets.size() > 0) { - mergeSmallColumnBatches(tx, pr, container, metaData, - smallBatchBuckets.asScala.filter(_._2.length > 1)) + if (smallBatchBuckets.nonEmpty) { + mergeSmallColumnBatches(pr, container, metaData, smallBatchBuckets) } } val stats = pr.getPrStats stats.setPRNumRowsInColumnBatches(rowsInColumnBatch) stats.setOffHeapSizeInBytes(offHeapSize) } else if (!isColumnTable && pr.getLocalMaxMemory > 0 && container.isRowBuffer) { - rolloverRowBuffers(pr) + rolloverTasks.computeIfAbsent(pr, rolloverRowBuffersTask) } } - } catch { - case t: Throwable => success = false; handleException(t) - } finally { - handleTransaction(cache, tx, context, success) } } + // Ensure max one background task per table + private val rolloverTasks = new ConcurrentHashMap[PartitionedRegion, Future[Unit]]() + private val mergeTasks = new ConcurrentHashMap[PartitionedRegion, Future[Unit]]() + private def minSizeForRollover(pr: PartitionedRegion): Int = math.max(pr.getColumnMaxDeltaRows >>> 3, pr.getColumnMinDeltaRows) @@ -352,138 +344,142 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService * Check if row buffers are large and have not been touched for a while * then roll it over into the column table */ - private def rolloverRowBuffers(pr: PartitionedRegion): Unit = { - val localPrimaries = pr.getDataStore.getAllLocalPrimaryBucketRegions - if ((localPrimaries ne null) && localPrimaries.size() > 0) { - val doRollover = new Predicate[BucketRegion] { - private val minModTime = pr.getCache.cacheTimeMillis() - delayMillis - - override def test(br: BucketRegion): Boolean = { - br.getLastModifiedTime <= minModTime && br.getRegionSize >= minSizeForRollover(pr) + private val rolloverRowBuffersTask = new JFunction[PartitionedRegion, Future[Unit]] { + override def apply(pr: PartitionedRegion): Future[Unit] = { + val localPrimaries = pr.getDataStore.getAllLocalPrimaryBucketRegions + if ((localPrimaries ne null) && localPrimaries.size() > 0) { + val doRollover = new Predicate[BucketRegion] { + private val minModTime = pr.getCache.cacheTimeMillis() - delayMillis + + override def test(br: BucketRegion): Boolean = { + br.getLastModifiedTime <= minModTime && br.getRegionSize >= minSizeForRollover(pr) + } } - } - val rolloverBuckets = localPrimaries.asScala.filter( - br => doRollover.test(br) && !br.columnBatchFlushLock.isWriteLocked) - // enqueue a job to roll over required row buffers into column table - // (each bucket will perform a last minute check before rollover inside lock) - if (rolloverBuckets.nonEmpty) { - // logInfo( - // s"SW:111: will rollover buckets for ${pr.getFullPath}: ${rolloverBuckets.map(_.getId)}") - implicit val executionContext = Utils.executionContext(pr.getGemFireCache) - Future(rolloverBuckets.foreach(bucket => withExceptionHandling( - bucket.createAndInsertColumnBatch(null, true, doRollover)))) - } + val rolloverBuckets = localPrimaries.asScala.filter( + br => doRollover.test(br) && !br.columnBatchFlushLock.isWriteLocked) + // enqueue a job to roll over required row buffers into column table + // (each bucket will perform a last minute check before rollover inside lock) + if (rolloverBuckets.nonEmpty) { + implicit val executionContext = Utils.executionContext(pr.getGemFireCache) + Future { + try { + rolloverBuckets.foreach(bucket => withExceptionHandling( + bucket.createAndInsertColumnBatch(null, true, doRollover))) + } finally { + rolloverTasks.remove(pr) + } + } + } else null + } else null } } - private val mergeTask = new AtomicReference[Future[Unit]]() - /** * Merge multiple column batches that are small in size in a bucket. * These can get created due to a small "tail" in bulk imports (large enough * to exceed minimal size that would have pushed them into row buffers), * or a time-based flush that tolerates small sized column batches due to - * [[rolloverRowBuffers]] or a forced flush of even smaller size for sample tables. + * [[rolloverRowBuffersTask]] or a forced flush of even smaller size for sample tables. */ - private def mergeSmallColumnBatches(tx: TXStateProxy, pr: PartitionedRegion, - container: GemFireContainer, metaData: ExternalTableMetaData, - smallBatches: mutable.Map[BucketRegion, mutable.ArrayBuffer[RegionEntry]]): Unit = { - if (mergeTask.get() ne null) return - // skip uncommitted entries for merge - val txState = tx.getTXStateForRead - // reverse iteration of entries so that remove does not change indices to be iterated - var skip = false - for ((br, entries) <- smallBatches; j <- (entries.length - 1) to 0) { - val entry = entries(j) - val re = entry match { - case e: AbstractRegionEntry => txState.getLocalEntry(pr, br, -1, e, false) - case _ => entry - } - if (re eq null) { - entries.remove(j) - if (entries.length <= 1) skip = true - } else if (re ne entry) { - entries(j) = re.asInstanceOf[RegionEntry] - } - } - // keep only batches with size > 1 - val batchBuckets = if (skip) smallBatches.filter(_._2.length > 1) else smallBatches - if (batchBuckets.nonEmpty) mergeTask.synchronized { - // synchronized instead of compareAndSet to avoid creating Future execution - if (mergeTask.get() ne null) return - logInfo( - s"Found small batches for ${pr.getName}: ${batchBuckets.map(_._2.map(_.getRawKey))}") - val cache = pr.getGemFireCache - implicit val executionContext = Utils.executionContext(cache) - mergeTask.set(Future(withExceptionHandling { - val tableName = container.getQualifiedTableName - val schema = metaData.schema.asInstanceOf[StructType] - val compileKey = tableName.concat(".MERGE_SMALL_BATCHES") - val gen = CodeGeneration.compileCode(compileKey, schema.fields, () => { - val schemaAttrs = Utils.schemaAttributes(schema) - val tableScan = ColumnTableScan(schemaAttrs, dataRDD = null, - otherRDDs = Nil, numBuckets = -1, partitionColumns = Nil, - partitionColumnAliases = Nil, baseRelation = null, schema, allFilters = Nil, - schemaAttrs, caseSensitive = true) - // reduce min delta row size to avoid going through rolloverRowBuffers again - val insertPlan = ColumnInsertExec(tableScan, Nil, Nil, - numBuckets = -1, isPartitioned = false, None, - (pr.getColumnBatchSize, minSizeForRollover(pr), metaData.compressionCodec), - tableName, onExecutor = true, schema, - metaData.externalStore.asInstanceOf[ExternalStore], useMemberVariables = false) - // now generate the code with the help of WholeStageCodegenExec - // this is only used for local code generation while its RDD semantics - // and related methods are all ignored - val (ctx, code) = ExternalStoreUtils.codeGenOnExecutor( - WholeStageCodegenExec(insertPlan), insertPlan) - val references = ctx.references - // also push the index of batchId reference at the end which can be - // used by caller to update the reference objects before execution - references += insertPlan.getBatchIdRef - (code, references.toArray) - }) - val references = gen._2.clone() - // full projection for the iterators - val numColumns = schema.length - val projection = (1 to numColumns).toArray - var success = false - var txState: TXStateProxy = null - var context: TXManagerImpl.TXContext = null - // for each bucket, create an iterator to scan and insert the result batches; - // a separate iterator is required because one ColumnInsertExec assumes a single batchId - for ((br, entries) <- batchBuckets) try { - success = false - // start a new transaction for each bucket - txState = null - txState = if (cache.snapshotEnabled) { - context = TXManagerImpl.getOrCreateTXContext() - cache.getCacheTransactionManager.beginTX(context, IsolationLevel.SNAPSHOT, null, null) - } else null - // update the bucketId as per the current bucket - val batchIdRef = references(references.length - 1).asInstanceOf[Int] - val bucketId = br.getId - references(batchIdRef + 1) = bucketId - val keys = entries.map(_.getRawKey.asInstanceOf[ColumnFormatKey]) - logInfo(s"Merging batches for ${pr.getName}:$bucketId :: $keys") - // no harm in passing a references array with an extra element at end - val iter = gen._1.generate(references).asInstanceOf[BufferedRowIterator] - // use the entries already determined for the iterator read by generated code - val batchIter = ColumnBatchIterator(br, entries.iterator, projection, context = null) - iter.init(bucketId, Array(Iterator[Any](new ResultSetTraversal(conn = null, stmt = null, - rs = null, context = null), batchIter).asInstanceOf[Iterator[InternalRow]])) - while (iter.hasNext) { - iter.next() // ignore result which is number of inserted rows + private def mergeSmallColumnBatches(pr: PartitionedRegion, container: GemFireContainer, + metaData: ExternalTableMetaData, smallBatches: mutable.ArrayBuffer[BucketRegion]): Unit = { + mergeTasks.computeIfAbsent(pr, new JFunction[PartitionedRegion, Future[Unit]] { + override def apply(pr: PartitionedRegion): Future[Unit] = { + logInfo(s"Found small batches in ${pr.getName}: ${smallBatches.map(_.getId)}") + val cache = pr.getGemFireCache + implicit val executionContext = Utils.executionContext(cache) + Future(withExceptionHandling({ + val tableName = container.getQualifiedTableName + val schema = metaData.schema.asInstanceOf[StructType] + val maxDeltaRows = pr.getColumnMaxDeltaRows + val compileKey = tableName.concat(".MERGE_SMALL_BATCHES") + val gen = CodeGeneration.compileCode(compileKey, schema.fields, () => { + val schemaAttrs = Utils.schemaAttributes(schema) + val tableScan = ColumnTableScan(schemaAttrs, dataRDD = null, + otherRDDs = Nil, numBuckets = -1, partitionColumns = Nil, + partitionColumnAliases = Nil, baseRelation = null, schema, allFilters = Nil, + schemaAttrs, caseSensitive = true) + // reduce min delta row size to avoid going through rolloverRowBuffers again + val insertPlan = ColumnInsertExec(tableScan, Nil, Nil, + numBuckets = -1, isPartitioned = false, None, + (pr.getColumnBatchSize, minSizeForRollover(pr), metaData.compressionCodec), + tableName, onExecutor = true, schema, + metaData.externalStore.asInstanceOf[ExternalStore], useMemberVariables = false) + // now generate the code with the help of WholeStageCodegenExec + // this is only used for local code generation while its RDD semantics + // and related methods are all ignored + val (ctx, code) = ExternalStoreUtils.codeGenOnExecutor( + WholeStageCodegenExec(insertPlan), insertPlan) + val references = ctx.references + // also push the index of batchId reference at the end which can be + // used by caller to update the reference objects before execution + references += insertPlan.getBatchIdRef + (code, references.toArray) + }) + val references = gen._2.clone() + // full projection for the iterators + val numColumns = schema.length + val projection = (1 to numColumns).toArray + var success = false + var tx: TXStateProxy = null + var context: TXManagerImpl.TXContext = null + // for each bucket, create an iterator to scan and insert the result batches; + // a separate iterator is required because one ColumnInsertExec assumes a single batchId + for (br <- smallBatches) try { + success = false + // start a new transaction for each bucket + tx = null + tx = if (cache.snapshotEnabled) { + context = TXManagerImpl.getOrCreateTXContext() + cache.getCacheTransactionManager.beginTX(context, IsolationLevel.SNAPSHOT, null, null) + } else null + // find the committed entries with small batches under the transaction + val bucketId = br.getId + val itr = new pr.PRLocalScanIterator(Collections.singleton(bucketId), + tx.getTXStateForRead, false /* forUpdate */ , false /* includeValues */ , + false /* fetchRemote */) + val entries = new mutable.ArrayBuffer[RegionEntry](2) + while (itr.hasNext) { + val re = itr.next().asInstanceOf[RegionEntry] + if (!re.isDestroyedOrRemoved) { + val key = re.getRawKey.asInstanceOf[ColumnFormatKey] + val batchRowCount = key.getColumnBatchRowCount(itr, re, schema.length) + // check if bucket has multiple small batches + if (key.getColumnIndex == ColumnFormatEntry.STATROW_COL_INDEX && + batchRowCount < maxDeltaRows) { + entries += re + } + } + } + itr.close() + if (entries.length > 1) { + // update the bucketId as per the current bucket + val batchIdRef = references(references.length - 1).asInstanceOf[Int] + references(batchIdRef + 1) = bucketId + val keys = entries.map(_.getRawKey.asInstanceOf[ColumnFormatKey]) + logInfo(s"Merging batches for ${pr.getName}:$bucketId :: $keys") + // no harm in passing a references array with an extra element at end + val iter = gen._1.generate(references).asInstanceOf[BufferedRowIterator] + // use the entries already determined for the iterator read by generated code + val batchIter = ColumnBatchIterator(br, entries.iterator, projection, context = null) + iter.init(bucketId, Array(Iterator[Any](new ResultSetTraversal( + conn = null, stmt = null, rs = null, context = null), batchIter) + .asInstanceOf[Iterator[InternalRow]])) + while (iter.hasNext) { + iter.next() // ignore result which is number of inserted rows + } + // now delete the keys that have been inserted above + logInfo(s"Deleting merged batches for ${pr.getName}:$bucketId :: $keys") + keys.foreach(ColumnDelta.deleteBatch(_, pr, numColumns)) + } + success = true + } catch { + case t: Throwable => handleException(t) + } finally { + handleTransaction(cache, tx, context, success) } - // now delete the keys that have been inserted above - logInfo(s"Deleting merged batches for ${pr.getName}:$bucketId :: $keys") - keys.foreach(ColumnDelta.deleteBatch(_, pr, numColumns)) - success = true - } catch { - case t: Throwable => handleException(t) - } finally { - handleTransaction(cache, txState, context, success) - } - })) - } + }, () => mergeTasks.remove(pr))) + } + }) } } diff --git a/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala b/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala index 153e4ab5c7..c63f445cee 100644 --- a/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala +++ b/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala @@ -185,9 +185,7 @@ object SmartConnectorRDDHelper { val netUrls = new ArrayBuffer[(String, String)](1) netUrls += host -> netUrl allNetUrls(bid) = netUrls - if (!availableNetUrls.containsKey(host)) { - availableNetUrls.put(host, netUrl) - } + availableNetUrls.putIfAbsent(host, netUrl) } else { // Save the bucket which does not have a neturl, // and later assign available ones to it. diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index 657b5fb67c..8d8f3195ca 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -660,7 +660,7 @@ object Utils { def toOpenHashMap[K, V](map: scala.collection.Map[K, V]): ObjectObjectHashMap[K, V] = { val m = ObjectObjectHashMap.withExpectedSize[K, V](map.size) - map.foreach(p => m.put(p._1, p._2)) + map.foreach(p => m.justPut(p._1, p._2)) m } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala b/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala index 4be32a06d3..51c3cf600a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import java.text.NumberFormat import java.util.Locale +import java.util.concurrent.atomic.AtomicLong import scala.collection.mutable @@ -38,11 +39,29 @@ object SnappyMetrics { */ val SPLIT_SUM_METRIC = "splitSum" + // in-built metric names for ColumnTableScan + val NUM_ROWS_DISK = "numRowsBufferDisk" + val NUM_BATCHES_DISK_FULL = "columnBatchesDiskFull" + val NUM_BATCHES_DISK_PARTIAL = "columnBatchesDiskPartial" + val NUM_BATCHES_REMOTE = "columnBatchesRemote" + + private val splitMetricId = new AtomicLong(0L) + + /** + * Get a new ID that can be used for [[createSplitSumMetric]]. + */ + def newSplitMetricId(): Long = math.abs(splitMetricId.getAndIncrement()) + /** * Create a metric to report multiple sums as a single metric. All metrics are combined * and displayed as comma-separated values against the name for "splitIndex" = 0. + * + * The ID should be a positive long common for all metrics that need to be displayed + * together and should be unique across all instances of the plan. + * The [[newSplitMetricId]] provides a convenient way to generate a new unique ID. */ - def createSplitSumMetric(sc: SparkContext, name: String, id: Int, splitIndex: Int): SQLMetric = { + def createSplitSumMetric(sc: SparkContext, name: String, + id: Long, splitIndex: Int): SQLMetric = { // The final result of this metric in physical operator UI may looks like: // data size total (min, med, max): // 100GB (100MB, 1GB, 10GB) @@ -61,7 +80,7 @@ object SnappyMetrics { val numberFormat = NumberFormat.getIntegerInstance(Locale.US) valueList.collect { case l if l ne null => numberFormat.format(l.toNativeArray.sum) - }.mkString(" / ") + }.mkString("|") } else SQLMetrics.stringValue(metricType, values.asInstanceOf[TLongArrayList].toNativeArray) } } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala index e17060ab57..1398bb4a61 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala @@ -34,6 +34,7 @@ import io.snappydata.collection.IntObjectHashMap import io.snappydata.thrift.common.BufferedBlob import org.apache.spark.memory.MemoryManagerCallback +import org.apache.spark.sql.execution.SnappyMetrics import org.apache.spark.sql.execution.columnar.encoding.{ColumnDecoder, ColumnDeleteDecoder, ColumnEncoding, UpdatedColumnDecoder, UpdatedColumnDecoderBase} import org.apache.spark.sql.execution.columnar.impl._ import org.apache.spark.sql.execution.metric.SQLMetric @@ -139,7 +140,7 @@ object ColumnBatchIterator { final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, statsEntries: Iterator[RegionEntry], bucketIds: java.util.Set[Integer], projection: Array[Int], fullScan: Boolean, context: TaskContext) - extends PRValuesIterator[ByteBuffer](container = null, region, bucketIds) { + extends PRValuesIterator[ByteBuffer](container = null, region, bucketIds) with Logging { if (region ne null) { assert(!region.getEnableOffHeapMemory, @@ -158,6 +159,7 @@ final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, private var diskBatchesFullMetric: SQLMetric = _ private var diskBatchesPartialMetric: SQLMetric = _ + private var remoteBatchesMetric: SQLMetric = _ override protected def createIterator(container: GemFireContainer, region: LocalRegion, tx: TXStateInterface): PRIterator = if (region ne null) { @@ -166,15 +168,21 @@ final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, java.util.Iterator[RegionEntry]] { override def apply(br: BucketRegion, numEntries: java.lang.Long): java.util.Iterator[RegionEntry] = { - if (statsEntries eq null) new ColumnFormatIterator(br, projection, fullScan, txState) - else new ColumnFormatStatsIterator(br, statsEntries, tx) + val itr = if (statsEntries eq null) { + new ColumnFormatIterator(br, projection, fullScan, txState) + } else new ColumnFormatStatsIterator(br, statsEntries, tx) + itr.setDiskMetric(diskBatchesFullMetric, isPartialMetric = false) + itr.setDiskMetric(diskBatchesPartialMetric, isPartialMetric = true) + itr } } val createRemoteIterator = new BiFunction[java.lang.Integer, PRIterator, java.util.Iterator[RegionEntry]] { override def apply(bucketId: Integer, iter: PRIterator): java.util.Iterator[RegionEntry] = { - new RemoteEntriesIterator(bucketId, projection, iter.getPartitionedRegion, tx) + val itr = new RemoteEntriesIterator(bucketId, projection, iter.getPartitionedRegion, tx) + itr.setMetric(remoteBatchesMetric) + itr } } val pr = region.asInstanceOf[PartitionedRegion] @@ -284,6 +292,7 @@ final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, currentColumns = new ArrayBuffer[ColumnFormatValue](math.max(1, releaseColumns())) currentVal = null currentDeltaStats = null + val itr = this.itr while (itr.hasNext) { val re = itr.next().asInstanceOf[RegionEntry] // the underlying ClusteredColumnIterator allows fetching entire projected @@ -324,8 +333,9 @@ final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, } override def setMetric(name: String, metric: SQLMetric): Boolean = name match { - case ColumnTableScan.NUM_BATCHES_DISK_PARTIAL => diskBatchesPartialMetric = metric; true - case ColumnTableScan.NUM_BATCHES_DISK_FULL => diskBatchesFullMetric = metric; true + case SnappyMetrics.NUM_BATCHES_DISK_FULL => diskBatchesFullMetric = metric; true + case SnappyMetrics.NUM_BATCHES_DISK_PARTIAL => diskBatchesPartialMetric = metric; true + case SnappyMetrics.NUM_BATCHES_REMOTE => remoteBatchesMetric = metric; true case _ => false } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala index 929fe9baec..cb0b4786f6 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala @@ -47,8 +47,8 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer} import org.apache.spark.sql.collection.Utils +import org.apache.spark.sql.execution.SnappyMetrics.{NUM_BATCHES_DISK_FULL, NUM_BATCHES_DISK_PARTIAL, NUM_BATCHES_REMOTE, NUM_ROWS_DISK} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.columnar.ColumnTableScan.{NUM_BATCHES_DISK_FULL, NUM_BATCHES_DISK_PARTIAL, NUM_ROWS_DISK} import org.apache.spark.sql.execution.columnar.encoding._ import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, ColumnDelta} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} @@ -102,26 +102,33 @@ final case class ColumnTableScan( override def getMetrics: Map[String, SQLMetric] = { if (sqlContext eq null) Map.empty - else super.getMetrics ++ Map( - "numRowsBuffer" -> SnappyMetrics.createSplitSumMetric(sparkContext, - "rows read from row buffer (total / disk)", 0, 0), - NUM_ROWS_DISK -> SnappyMetrics.createSplitSumMetric(sparkContext, - "number of rows on disk from row buffer", 0, 1), - "columnBatchesSeen" -> SnappyMetrics.createSplitSumMetric(sparkContext, - "column batches read (total / disk-partial / disk-full)", 1, 0), - NUM_BATCHES_DISK_PARTIAL -> SnappyMetrics.createSplitSumMetric(sparkContext, - "column batches read from disk (partial)", 1, 1), - NUM_BATCHES_DISK_FULL -> SnappyMetrics.createSplitSumMetric(sparkContext, - "column batches read from disk (full)", 1, 2), - "updatedColumnCount" -> SnappyMetrics.createSplitSumMetric(sparkContext, - "deltas (updated columns / batches having deletes)", 2, 0), - "deletedBatchCount" -> SnappyMetrics.createSplitSumMetric(sparkContext, - "column batches having deletes", 2, 1), - "columnBatchesSkipped" -> SQLMetrics.createMetric(sparkContext, - "column batches skipped by the predicate")) ++ ( - if (otherRDDs.isEmpty) Map.empty - else Map("numRowsOtherRDDs" -> SQLMetrics.createMetric(sparkContext, - "number of output rows from other RDDs"))) + else { + val id0 = SnappyMetrics.newSplitMetricId() + val id1 = SnappyMetrics.newSplitMetricId() + val id2 = SnappyMetrics.newSplitMetricId() + super.getMetrics ++ Map( + "numRowsBuffer" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "row buffer reads (total)", id0, 1), + NUM_ROWS_DISK -> SnappyMetrics.createSplitSumMetric(sparkContext, + "row buffer reads (disk|total)", id0, 0), + "columnBatchesSeen" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "batches (total)", id1, 3), + NUM_BATCHES_DISK_FULL -> SnappyMetrics.createSplitSumMetric(sparkContext, + "batches (disk)", id1, 2), + NUM_BATCHES_DISK_PARTIAL -> SnappyMetrics.createSplitSumMetric(sparkContext, + "batches (disk-partial)", id1, 1), + NUM_BATCHES_REMOTE -> SnappyMetrics.createSplitSumMetric(sparkContext, + "batches (remote|disk-partial|disk|total)", id1, 0), + "updatedColumnCount" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "deltas (updated columns)", id2, 1), + "deletedBatchCount" -> SnappyMetrics.createSplitSumMetric(sparkContext, + "deltas (deletes|updated columns)", id2, 0), + "columnBatchesSkipped" -> SQLMetrics.createMetric(sparkContext, + "batches skipped by predicates")) ++ ( + if (otherRDDs.isEmpty) Map.empty + else Map("numRowsOtherRDDs" -> SQLMetrics.createMetric(sparkContext, + "number of output rows from other RDDs"))) + } } override def metricTerm(ctx: CodegenContext, name: String): String = @@ -187,6 +194,7 @@ final case class ColumnTableScan( val numRowsBufferDisk = metricTerm(ctx, NUM_ROWS_DISK) val numBatchesDiskPartial = metricTerm(ctx, NUM_BATCHES_DISK_PARTIAL) val numBatchesDiskFull = metricTerm(ctx, NUM_BATCHES_DISK_FULL) + val numBatchesRemote = metricTerm(ctx, NUM_BATCHES_REMOTE) val numRowsOther = if (otherRDDs.isEmpty) null else metricTerm(ctx, "numRowsOtherRDDs") val embedded = (baseRelation eq null) || @@ -349,6 +357,7 @@ final case class ColumnTableScan( $iteratorWithMetricsClass mIter = ($iteratorWithMetricsClass)$colInput; mIter.setMetric("$NUM_BATCHES_DISK_PARTIAL", $numBatchesDiskPartial); mIter.setMetric("$NUM_BATCHES_DISK_FULL", $numBatchesDiskFull); + mIter.setMetric("$NUM_BATCHES_REMOTE", $numBatchesRemote); } """ } @@ -825,10 +834,6 @@ final case class ColumnTableScan( object ColumnTableScan extends Logging { - val NUM_ROWS_DISK = "numRowsBufferDisk" - val NUM_BATCHES_DISK_PARTIAL = "columnBatchesDiskPartial" - val NUM_BATCHES_DISK_FULL = "columnBatchesDiskFull" - def generateStatPredicate(ctx: CodegenContext, isColumnTable: Boolean, schemaAttrs: Seq[AttributeReference], allFilters: Seq[Expression], numRowsTerm: String, metricTerm: (CodegenContext, String) => String, metricAdd: String => String): String = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala index 2118116918..cdf16ae92d 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ClusteredColumnIterator.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.execution.columnar.impl import com.gemstone.gemfire.internal.cache.RegionEntry import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator +import org.apache.spark.sql.execution.metric.SQLMetric + /** - * Base trait for iterators that are capable of reading and returning + * Base class for iterators that are capable of reading and returning * the entire set of columns of a column batch. These can be local region * iterators or those fetching entries from remote nodes. */ @@ -33,3 +35,21 @@ abstract class ClusteredColumnIterator extends CloseableIterator[RegionEntry] { */ def getColumnValue(column: Int): AnyRef } + +/** + * Base class for local [[ClusteredColumnIterator]]s that can read from memory or disk. + */ +abstract class ClusteredDiskIterator extends ClusteredColumnIterator { + + protected final var diskBatchesFull: SQLMetric = _ + protected final var diskBatchesPartial: SQLMetric = _ + protected final var checkDiskRead = false + + /** + * Set metrics to track disk reads by this iterator. + */ + def setDiskMetric(diskRead: SQLMetric, isPartialMetric: Boolean): Unit = { + checkDiskRead = true + if (isPartialMetric) this.diskBatchesPartial = diskRead else this.diskBatchesFull = diskRead + } +} diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala index f8ac46a388..842bd6f077 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala @@ -32,6 +32,7 @@ import io.snappydata.collection.LongObjectHashMap import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.columnar.encoding.BitSet import org.apache.spark.sql.execution.columnar.impl.ColumnFormatEntry._ +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.unsafe.Platform /** @@ -46,7 +47,7 @@ import org.apache.spark.unsafe.Platform */ final class ColumnFormatIterator(baseRegion: LocalRegion, projection: Array[Int], fullScan: Boolean, txState: TXState) - extends ClusteredColumnIterator with DiskRegionIterator { + extends ClusteredDiskIterator with DiskRegionIterator { type MapValueIterator = CustomEntryConcurrentHashMap[AnyRef, AbstractRegionEntry]#ValueIterator @@ -60,6 +61,7 @@ final class ColumnFormatIterator(baseRegion: LocalRegion, projection: Array[Int] private var diskEnumerator: DiskBlockSorter#ReaderIdEnumerator = _ private var currentDiskBatch: DiskMultiColumnBatch = _ private var nextDiskBatch: DiskMultiColumnBatch = _ + private val currentDiskEntryMap = LongObjectHashMap.withExpectedSize[AnyRef](16) /** * The current set of in-memory batches being iterated. @@ -158,7 +160,7 @@ final class ColumnFormatIterator(baseRegion: LocalRegion, projection: Array[Int] val map = inMemoryBatches.get(inMemoryBatchIndex) map.getGlobalState.asInstanceOf[RegionEntry] } else if (nextDiskBatch ne null) { - if (currentDiskBatch ne null) currentDiskBatch.release() + releaseCurrentBatch() currentDiskBatch = nextDiskBatch nextDiskBatch = diskEnumerator.nextElement().asInstanceOf[DiskMultiColumnBatch] currentDiskBatch.getEntry @@ -172,13 +174,31 @@ final class ColumnFormatIterator(baseRegion: LocalRegion, projection: Array[Int] val column = columnIndex & 0xffffffffL if (entryIterator ne null) inMemoryBatches.get(inMemoryBatchIndex).get(column) else if (columnIndex == DELTA_STATROW_COL_INDEX) currentDiskBatch.getDeltaStatsValue - else currentDiskBatch.entryMap.get(column) + else { + currentDiskBatch.fillEntryMap(currentDiskEntryMap, diskBatchesFull, + diskBatchesPartial, checkDiskRead) + currentDiskEntryMap.get(column) + } } override def close(): Unit = { - if (currentDiskBatch ne null) { - currentDiskBatch.release() - currentDiskBatch = null + releaseCurrentBatch() + currentDiskBatch = null + } + + private def releaseCurrentBatch(): Unit = { + val entryMap = this.currentDiskEntryMap + if (entryMap.size() > 0) { + if (GemFireCacheImpl.hasNewOffHeap) entryMap.forEachWhile(new LongObjPredicate[AnyRef] { + override def test(i: Long, v: AnyRef): Boolean = { + v match { + case s: SerializedDiskBuffer => s.release() + case _ => + } + true + } + }) + entryMap.clear() } } @@ -316,21 +336,22 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, private var arrayIndex: Int = _ private var faultIn: Boolean = _ - private var closing: Boolean = _ // track delta stats separately since it is required for stats filtering // and should not lead to other columns getting read from disk (or worse faulted in) private var deltaStatsEntry: RegionEntry = _ - private[impl] lazy val entryMap: LongObjectHashMap[AnyRef] = { - if (closing) null - else { + private[impl] def fillEntryMap(map: LongObjectHashMap[AnyRef], diskBatchesFull: SQLMetric, + diskBatchesPartial: SQLMetric, checkDiskRead: Boolean): Unit = { + val numEntries = arrayIndex + if (map.size() == 0 && numEntries > 0) { // read all the entries in this column batch to fault them in or read without // fault-in at this point to build the temporary column to value map for this batch - val map = LongObjectHashMap.withExpectedSize[AnyRef](arrayIndex) + // count number of entries on disk if required + var numOnDisk = 0 var i = 0 - while (i < arrayIndex) { - val entry = diskEntries(i) - val re = entry.asInstanceOf[RegionEntry] + while (i < numEntries) { + val re = diskEntries(i).asInstanceOf[RegionEntry] + if (checkDiskRead && re.isValueNull) numOnDisk += 1 val v = if (faultIn) { val v = re.getValue(region) if (GemFireCacheImpl.hasNewOffHeap) v match { @@ -343,7 +364,11 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, i += 1 } diskEntries = null - map + if (checkDiskRead) { + if (numOnDisk == numEntries) { + if (diskBatchesFull ne null) diskBatchesFull.add(1) + } else if (diskBatchesPartial ne null) diskBatchesPartial.add(1) + } } } @@ -371,13 +396,14 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, } def finish(): Unit = { - if (arrayIndex > 0) { + val numEntries = arrayIndex + if (numEntries > 0) { // generally small size to sort so will be done efficiently in-place by the normal // sorter and hence not using the GemXD TimSort that reuses potentially large arrays - java.util.Arrays.sort(diskEntries, 0, arrayIndex, DiskEntryPage.DEPComparator.instance) + java.util.Arrays.sort(diskEntries, 0, numEntries, DiskEntryPage.DEPComparator.instance) // replace the DiskEntryPage objects with RegionEntry to release the extra memory var i = 0 - while (i < arrayIndex) { + while (i < numEntries) { val diskEntry = diskEntries(i).asInstanceOf[DiskEntryPage] // set the minimum position as the one to be used for this multi-column batch if (i == 0) setPosition(diskEntry.getOplogId, diskEntry.getOffset) @@ -390,26 +416,8 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, override protected def readEntryValue(): AnyRef = { // mark the entryMap for fault-in faultIn = true - closing = false super.readEntryValue() } - - private[impl] def release(): Unit = { - closing = true - val entryMap = this.entryMap - if ((entryMap ne null) && entryMap.size() > 0) { - if (GemFireCacheImpl.hasNewOffHeap) entryMap.forEachWhile(new LongObjPredicate[AnyRef] { - override def test(i: Long, v: AnyRef): Boolean = { - v match { - case s: SerializedDiskBuffer => s.release() - case _ => - } - true - } - }) - entryMap.clear() - } - } } /** @@ -419,7 +427,7 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, * should use the normal ColumnFormatIterator. */ final class ColumnFormatStatsIterator(bucketRegion: BucketRegion, - statsEntries: Iterator[RegionEntry], tx: TXStateInterface) extends ClusteredColumnIterator { + statsEntries: Iterator[RegionEntry], tx: TXStateInterface) extends ClusteredDiskIterator { try { bucketRegion.checkReadiness() diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala index 087485dbf1..14e3fc5ba7 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/RemoteEntriesIterator.scala @@ -32,6 +32,7 @@ import com.pivotal.gemfirexd.internal.engine.sql.execute.GemFireResultSet import io.snappydata.collection.IntObjectHashMap import org.apache.spark.sql.execution.columnar.impl.ColumnFormatEntry._ +import org.apache.spark.sql.execution.metric.SQLMetric /** * A [[ClusteredColumnIterator]] that fetches entries from a remote bucket. @@ -163,6 +164,13 @@ final class RemoteEntriesIterator(bucketId: Int, projection: Array[Int], private var currentDeltaStats: AnyRef = _ private val currentValueMap = IntObjectHashMap.withExpectedSize[AnyRef](8) + private var remoteBatches: SQLMetric = _ + + /** + * Set metrics to track remote reads by this iterator. + */ + def setMetric(remoteBatches: SQLMetric): Unit = this.remoteBatches = remoteBatches + private def fetchUsingGetAll(keys: Array[AnyRef]): Seq[(AnyRef, AnyRef)] = { val msg = new GetAllExecutorMessage(pr, keys, null, null, null, null, null, null, tx, null, false, false) @@ -209,6 +217,7 @@ final class RemoteEntriesIterator(bucketId: Int, projection: Array[Int], fetchUsingGetAll(fetchKeys).foreach { case (k: ColumnFormatKey, v) => currentValueMap.justPut(k.columnIndex, v) } + if (remoteBatches ne null) remoteBatches.add(1) } currentValueMap.get(column) } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala index 7272ae3665..c457d148ab 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala @@ -29,6 +29,7 @@ import com.gemstone.gemfire.internal.cache.{BucketRegion, EntryEventImpl, Extern import com.gemstone.gemfire.internal.shared.FetchRequest import com.gemstone.gemfire.internal.snappy.memory.MemoryManagerStats import com.gemstone.gemfire.internal.snappy.{CallbackFactoryProvider, ColumnTableEntry, StoreCallbacks, UMMMemoryTracker} +import com.google.common.cache.Cache import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.access.GemFireTransaction import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils @@ -621,6 +622,14 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable override def clearConnectionPools(): Unit = { ConnectionPool.clear() } + + override def clearCodegenCaches(): Unit = { + CodeGeneration.clearAllCache() + val cacheField = CodeGenerator.getClass.getDeclaredFields.find(_.getName.endsWith("cache")).get + cacheField.setAccessible(true) + val cache = cacheField.get(CodeGenerator).asInstanceOf[Cache[_, _]] + cache.invalidateAll() + } } trait StoreCallback extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index 911467f970..4b57786bcc 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -42,10 +42,10 @@ import org.apache.spark.serializer.ConnectionPropertiesSerializer import org.apache.spark.sql.SnappySession import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.collection.MultiBucketExecutorPartition -import org.apache.spark.sql.execution.columnar.{ColumnTableScan, ExternalStoreUtils, ResultSetIterator} +import org.apache.spark.sql.execution.columnar.{ExternalStoreUtils, ResultSetIterator} import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.sources.StoreDataSourceStrategy.translateToFilter -import org.apache.spark.sql.execution.{IteratorWithMetrics, RDDKryo, SecurityUtils} +import org.apache.spark.sql.execution.{IteratorWithMetrics, RDDKryo, SecurityUtils, SnappyMetrics} import org.apache.spark.sql.sources.JdbcExtendedUtils.quotedName import org.apache.spark.sql.sources._ import org.apache.spark.{Partition, TaskContext} @@ -304,7 +304,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, val dataItr = itr.map(r => if (r.hasByteArrays) r.getRowByteArrays(null) else r.getRowBytes(null): AnyRef).asJava val rs = new RawStoreResultSet(dataItr, container, container.getCurrentRowFormatter) - new ResultSetTraversal(conn = null, stmt = null, rs, context) + new ResultSetTraversal(conn = null, stmt = null, rs, context, Some(itr)) } else itr } else { val (conn, stmt, rs) = computeResultSet(thePart, context) @@ -419,13 +419,19 @@ class RowFormatScanRDD(@transient val session: SnappySession, * This is primarily intended to be used for cleanup. */ final class ResultSetTraversal(conn: Connection, - stmt: Statement, val rs: ResultSet, context: TaskContext) - extends ResultSetIterator[Void](conn, stmt, rs, context) { + stmt: Statement, val rs: ResultSet, context: TaskContext, + source: Option[IteratorWithMetrics[_]] = None) + extends ResultSetIterator[Void](conn, stmt, rs, context) with IteratorWithMetrics[Void] { lazy val defaultCal: GregorianCalendar = ClientSharedData.getDefaultCleanCalendar override protected def getCurrentValue: Void = null + + override def setMetric(name: String, metric: SQLMetric): Boolean = source match { + case Some(s) => s.setMetric(name, metric) + case None => false + } } final class CompactExecRowIteratorOnRS(conn: Connection, @@ -493,8 +499,9 @@ final class CompactExecRowIteratorOnScan(container: GemFireContainer, while (itr.hasNext) { val rl = itr.next().asInstanceOf[RowLocation] val owner = itr.getHostedBucketRegion - if ((owner ne null) || rl.isInstanceOf[NonLocalRegionEntry]) { - val valueWasNull = (diskRowsMetric ne null) && rl.isValueNull + val isNonLocalEntry = rl.isInstanceOf[NonLocalRegionEntry] + if ((owner ne null) || isNonLocalEntry) { + val valueWasNull = !isNonLocalEntry && (diskRowsMetric ne null) && rl.isValueNull if (faultIn) { if (RegionEntryUtils.fillRowFaultInOptimized(container, owner, rl, currentVal)) { if (valueWasNull) diskRowsMetric.add(1) @@ -511,7 +518,7 @@ final class CompactExecRowIteratorOnScan(container: GemFireContainer, } override def setMetric(name: String, metric: SQLMetric): Boolean = { - if (name == ColumnTableScan.NUM_ROWS_DISK) { + if (name == SnappyMetrics.NUM_ROWS_DISK) { diskRowsMetric = metric true } else false diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala index bb345e8758..1582aafcbc 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ui/SnappySQLListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.ui import java.util.AbstractMap.SimpleEntry -import java.util.function.BiFunction +import java.util.function.{Function => JFunction} import scala.collection.JavaConverters._ import scala.collection.mutable @@ -186,6 +186,14 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { } } + private def expandBuffer(b: mutable.ArrayBuffer[TLongArrayList], size: Int): Unit = { + var i = b.length + while (i < size) { + b += null + i += 1 + } + } + private def mergeAccumulatorUpdates( accumulatorUpdates: Seq[(Long, Any)], metricTypeFunc: Long => String): Map[Long, String] = { @@ -202,38 +210,23 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { val splitIndex = metricType.indexOf('_') val key = metricType.substring(0, splitIndex) val index = metricType.substring(splitIndex + 1).toInt - accumulatorMap.compute(key, new BiFunction[Any, MapValue, MapValue] { - private def expandBuffer(b: mutable.ArrayBuffer[TLongArrayList], size: Int): Unit = { - var i = b.length - while (i < size) { - b += null - i += 1 - } - } - - override def apply(key: Any, v: MapValue): MapValue = { - val mapValue = if (v ne null) v - else new MapValue(new mutable.ArrayBuffer[TLongArrayList](math.max(index + 1, 4)), 0L) - val valueList = mapValue.getKey.asInstanceOf[mutable.ArrayBuffer[TLongArrayList]] - expandBuffer(valueList, index + 1) - val values = valueList(index) match { - case null => val l = new TLongArrayList(4); valueList(index) = l; l - case l => l - } - values.add(value.asInstanceOf[Long]) - if (index == 0) mapValue.setValue(accumulatorId) - mapValue - } + val mapValue = accumulatorMap.computeIfAbsent(key, new JFunction[Any, MapValue] { + override def apply(k: Any): MapValue = + new MapValue(new mutable.ArrayBuffer[TLongArrayList](math.max(index + 1, 4)), 0L) }) + val valueList = mapValue.getKey.asInstanceOf[mutable.ArrayBuffer[TLongArrayList]] + expandBuffer(valueList, index + 1) + val values = valueList(index) match { + case null => val l = new TLongArrayList(4); valueList(index) = l; l + case l => l + } + values.add(value.asInstanceOf[Long]) + if (index == 0) mapValue.setValue(accumulatorId) } else { - accumulatorMap.compute(accumulatorId, new BiFunction[Any, MapValue, MapValue] { - override def apply(key: Any, v: MapValue): MapValue = { - val mapValue = if (v ne null) v - else new MapValue(new TLongArrayList(4), metricType) - mapValue.getKey.asInstanceOf[TLongArrayList].add(value.asInstanceOf[Long]) - mapValue - } + val mapValue = accumulatorMap.computeIfAbsent(accumulatorId, new JFunction[Any, MapValue] { + override def apply(k: Any): MapValue = new MapValue(new TLongArrayList(4), metricType) }) + mapValue.getKey.asInstanceOf[TLongArrayList].add(value.asInstanceOf[Long]) } } // now create a map on accumulatorId and the values (which are either a diff --git a/core/src/main/scala/org/apache/spark/sql/hive/SnappyStoreHiveCatalog.scala b/core/src/main/scala/org/apache/spark/sql/hive/SnappyStoreHiveCatalog.scala index 4b2e653161..79f75c883a 100644 --- a/core/src/main/scala/org/apache/spark/sql/hive/SnappyStoreHiveCatalog.scala +++ b/core/src/main/scala/org/apache/spark/sql/hive/SnappyStoreHiveCatalog.scala @@ -53,8 +53,7 @@ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.collection.{ToolsCallbackInit, Utils} import org.apache.spark.sql.execution.columnar.ExternalStoreUtils.CaseInsensitiveMutableHashMap -import org.apache.spark.sql.execution.columnar.impl.{DefaultSource => ColumnSource} -import org.apache.spark.sql.execution.columnar.impl.IndexColumnFormatRelation +import org.apache.spark.sql.execution.columnar.impl.{IndexColumnFormatRelation, DefaultSource => ColumnSource} import org.apache.spark.sql.execution.columnar.{ExternalStoreUtils, JDBCAppendableRelation} import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation} import org.apache.spark.sql.hive.SnappyStoreHiveCatalog._ @@ -127,7 +126,7 @@ class SnappyStoreHiveCatalog(externalCatalog: SnappyExternalCatalog, case _ => // Initialize default database if it doesn't already exist val defaultDbDefinition = - CatalogDatabase(defaultName, "app database", sqlConf.warehousePath, Map()) + CatalogDatabase(defaultName, "default user database", sqlConf.warehousePath, Map()) externalCatalog.createDatabase(defaultDbDefinition, ignoreIfExists = true) client.setCurrentDatabase(defaultName) } @@ -922,7 +921,7 @@ class SnappyStoreHiveCatalog(externalCatalog: SnappyExternalCatalog, * Create a metastore function in the database specified in `funcDefinition`. * If no such database is specified, create it in the current database. * If the specified database is not present in catalog, create that database. - * @TODO Ideally create schema from gfxd should get routed to create the database in + * @todo Ideally create schema from gfxd should get routed to create the database in * the Hive catalog. */ override def createFunction(funcDefinition: CatalogFunction, ignoreIfExists: Boolean): Unit = { diff --git a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala index ae1992b194..2006236a07 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala @@ -221,9 +221,10 @@ object StoreUtils { private def allocateBucketsToPartitions(session: SnappySession, region: PartitionedRegion, preferPrimaries: Boolean): Array[Partition] = { + type ServerBucket = (Option[BlockAndExecutorId], mutable.ArrayBuffer[Int]) val numTotalBuckets = region.getTotalNumberOfBuckets val serverToBuckets = ObjectObjectHashMap.withExpectedSize[InternalDistributedMember, - (Option[BlockAndExecutorId], mutable.ArrayBuffer[Int])](4) + ServerBucket](4) val adviser = region.getRegionAdvisor for (p <- 0 until numTotalBuckets) { var prefNode = if (preferPrimaries) region.getOrCreateNodeForBucketWrite(p, null) @@ -247,13 +248,11 @@ object StoreUtils { } }) } - val buckets = serverToBuckets.get(prefNode) match { - case null => - val buckets = new mutable.ArrayBuffer[Int]() - serverToBuckets.put(prefNode, prefBlockId -> buckets) - buckets - case b => b._2 - } + val buckets = serverToBuckets.computeIfAbsent(prefNode, + new java.util.function.Function[InternalDistributedMember, ServerBucket] { + override def apply(n: InternalDistributedMember): ServerBucket = + prefBlockId -> new mutable.ArrayBuffer[Int]() + })._2 buckets += p } // marker array to check that all buckets have been allocated diff --git a/store b/store index 4838032a21..93602d458f 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 4838032a21daf03cad933cac8fb728426c99c759 +Subproject commit 93602d458f8c4c9f95adf2e780dbee7cd83152ad From a1dadd9a9977526ae8b7dde6bc55aefdb4c74939 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 5 Jun 2018 03:31:09 +0530 Subject: [PATCH 06/19] minor change --- .../org/apache/spark/sql/execution/columnar/ColumnBatch.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala index 1398bb4a61..611d6f8258 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala @@ -140,7 +140,7 @@ object ColumnBatchIterator { final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, statsEntries: Iterator[RegionEntry], bucketIds: java.util.Set[Integer], projection: Array[Int], fullScan: Boolean, context: TaskContext) - extends PRValuesIterator[ByteBuffer](container = null, region, bucketIds) with Logging { + extends PRValuesIterator[ByteBuffer](container = null, region, bucketIds) { if (region ne null) { assert(!region.getEnableOffHeapMemory, From cf1eb841266a7949dc88ffdcaef1e7254a96e673 Mon Sep 17 00:00:00 2001 From: Trilok Khairnar Date: Tue, 5 Jun 2018 23:19:27 +0530 Subject: [PATCH 07/19] Fixing precheckin failure in SNAP-2365 and and adding similar fix for replicated table --- .../TPCHColumnPartitionedTable.scala | 25 ++++++++++--------- .../benchmark/TPCHReplicatedTable.scala | 11 +++++--- .../TPCH/{customer.tbl.1 => customer.tbl} | 0 .../TPCH/{lineitem.tbl.1 => lineitem.tbl} | 0 .../TPCH/{orders.tbl.1 => orders.tbl} | 0 .../resources/TPCH/{part.tbl.1 => part.tbl} | 0 .../TPCH/{partsupp.tbl.1 => partsupp.tbl} | 0 .../TPCH/{supplier.tbl.1 => supplier.tbl} | 0 8 files changed, 21 insertions(+), 15 deletions(-) rename tests/common/src/main/resources/TPCH/{customer.tbl.1 => customer.tbl} (100%) rename tests/common/src/main/resources/TPCH/{lineitem.tbl.1 => lineitem.tbl} (100%) rename tests/common/src/main/resources/TPCH/{orders.tbl.1 => orders.tbl} (100%) rename tests/common/src/main/resources/TPCH/{part.tbl.1 => part.tbl} (100%) rename tests/common/src/main/resources/TPCH/{partsupp.tbl.1 => partsupp.tbl} (100%) rename tests/common/src/main/resources/TPCH/{supplier.tbl.1 => supplier.tbl} (100%) diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala index 948ccf53d0..33b9660f80 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHColumnPartitionedTable.scala @@ -132,9 +132,10 @@ object TPCHColumnPartitionedTable { if (isParquet) { orderDF = sqlContext.read.format("parquet").load(s"$path/parquet_orders_$i") } else { - var stage = ""; // apply a tbl.i suffix to table filename only when data is loaded in more than one stages. + // apply a tbl.i suffix to table filename only when data is loaded in more than one stages. + var stage = "" if (numberOfLoadingStages > 1) { - stage = s".$i"; + stage = s".$i" } val orderData = sc.textFile(s"$path/orders.tbl$stage") val orderReadings = orderData.map(s => s.split('|')).map( @@ -221,9 +222,9 @@ object TPCHColumnPartitionedTable { if (isParquet) { lineItemDF = sqlContext.read.format("parquet").load(s"$path/parquet_lineitem_$i") } else { - var stage = ""; + var stage = "" if (numberOfLoadingStages > 1) { - stage = s".$i"; + stage = s".$i" } val lineItemData = sc.textFile(s"$path/lineitem.tbl$stage") val lineItemReadings = lineItemData.map(s => s.split('|')).map(s => TPCHTableSchema @@ -311,9 +312,9 @@ object TPCHColumnPartitionedTable { if (isParquet) { customerDF = sqlContext.read.format("parquet").load(s"$path/parquet_customer_$i") } else { - var stage = ""; + var stage = "" if (numberOfLoadingStages > 1) { - stage = s".$i"; + stage = s".$i" } val customerData = sc.textFile(s"$path/customer.tbl$stage") val customerReadings = customerData.map(s => s.split('|')).map(s => TPCHTableSchema @@ -378,9 +379,9 @@ object TPCHColumnPartitionedTable { if (isParquet) { partDF = sqlContext.read.format("parquet").load(s"$path/parquet_part_$i") } else { - var stage = ""; + var stage = "" if (numberOfLoadingStages > 1) { - stage = s".$i"; + stage = s".$i" } val partData = sc.textFile(s"$path/part.tbl$stage") val partReadings = partData.map(s => s.split('|')).map(s => TPCHTableSchema.parsePartRow(s)) @@ -442,9 +443,9 @@ object TPCHColumnPartitionedTable { if (isParquet) { partSuppDF = sqlContext.read.format("parquet").load(s"$path/parquet_partsupp_$i") } else { - var stage = ""; + var stage = "" if (numberOfLoadingStages > 1) { - stage = s".$i"; + stage = s".$i" } val partSuppData = sc.textFile(s"$path/partsupp.tbl$stage") val partSuppReadings = partSuppData.map(s => s.split('|')).map(s => TPCHTableSchema @@ -587,9 +588,9 @@ object TPCHColumnPartitionedTable { if (isParquet && new File(parquetDir).exists()) { suppDF = sqlContext.read.format("parquet").load(parquetDir) } else { - var stage = ""; + var stage = "" if (numberOfLoadingStages > 1) { - stage = s".$i"; + stage = s".$i" } val suppData = sc.textFile(s"$path/supplier.tbl$stage") val suppReadings = suppData.map(s => s.split('|')).map(s => TPCHTableSchema diff --git a/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala b/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala index 649841949d..fbb3172282 100644 --- a/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala +++ b/cluster/src/test/scala/io/snappydata/benchmark/TPCHReplicatedTable.scala @@ -119,14 +119,19 @@ object TPCHReplicatedTable { } def createPopulateSupplierTable(usingOptionString: String, sqlContext: SQLContext, path: String, - isSnappy: Boolean, loadPerfPrintStream: PrintStream = null, numberOfLoadingStage : Int = 1) + isSnappy: Boolean, loadPerfPrintStream: PrintStream = null, numberOfLoadingStages : Int = 1) : Unit = { val sc = sqlContext.sparkContext val startTime = System.currentTimeMillis() var unionSupplierDF : DataFrame = null - for(i <- 1 to numberOfLoadingStage) { - val supplierData = sc.textFile(s"$path/supplier.tbl.$i") + for(i <- 1 to numberOfLoadingStages) { + // apply a tbl.i suffix to table filename only when data is loaded in more than one stages. + var stage = "" + if (numberOfLoadingStages > 1) { + stage = s".$i" + } + val supplierData = sc.textFile(s"$path/supplier.tbl$stage") val supplierReadings = supplierData.map(s => s.split('|')).map(s => TPCHTableSchema .parseSupplierRow(s)) val supplierDF = sqlContext.createDataFrame(supplierReadings) diff --git a/tests/common/src/main/resources/TPCH/customer.tbl.1 b/tests/common/src/main/resources/TPCH/customer.tbl similarity index 100% rename from tests/common/src/main/resources/TPCH/customer.tbl.1 rename to tests/common/src/main/resources/TPCH/customer.tbl diff --git a/tests/common/src/main/resources/TPCH/lineitem.tbl.1 b/tests/common/src/main/resources/TPCH/lineitem.tbl similarity index 100% rename from tests/common/src/main/resources/TPCH/lineitem.tbl.1 rename to tests/common/src/main/resources/TPCH/lineitem.tbl diff --git a/tests/common/src/main/resources/TPCH/orders.tbl.1 b/tests/common/src/main/resources/TPCH/orders.tbl similarity index 100% rename from tests/common/src/main/resources/TPCH/orders.tbl.1 rename to tests/common/src/main/resources/TPCH/orders.tbl diff --git a/tests/common/src/main/resources/TPCH/part.tbl.1 b/tests/common/src/main/resources/TPCH/part.tbl similarity index 100% rename from tests/common/src/main/resources/TPCH/part.tbl.1 rename to tests/common/src/main/resources/TPCH/part.tbl diff --git a/tests/common/src/main/resources/TPCH/partsupp.tbl.1 b/tests/common/src/main/resources/TPCH/partsupp.tbl similarity index 100% rename from tests/common/src/main/resources/TPCH/partsupp.tbl.1 rename to tests/common/src/main/resources/TPCH/partsupp.tbl diff --git a/tests/common/src/main/resources/TPCH/supplier.tbl.1 b/tests/common/src/main/resources/TPCH/supplier.tbl similarity index 100% rename from tests/common/src/main/resources/TPCH/supplier.tbl.1 rename to tests/common/src/main/resources/TPCH/supplier.tbl From c8582736b0f1d8ee4c70ae72b4573475815210fa Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 29 Jun 2018 03:08:30 +0530 Subject: [PATCH 08/19] coarse container locking to fix rollover/merge when running in parallel with update/delete --- .../spark/memory/SnappyStorageEvictor.scala | 32 ++-- .../test/scala/io/snappydata/QueryTest.scala | 12 +- .../memory/SnappyStorageEvictorSuite.scala | 2 +- .../sql/store/ColumnUpdateDeleteTest.scala | 8 +- .../SnappyTableStatsProviderService.scala | 161 ++++++++++-------- .../apache/spark/sql/CachedDataFrame.scala | 2 +- .../apache/spark/sql/collection/Utils.scala | 30 +++- .../spark/sql/execution/ExistingPlans.scala | 22 ++- .../spark/sql/execution/TableExec.scala | 2 +- .../aggregate/SnappyHashAggregateExec.scala | 19 +-- .../sql/execution/columnar/ColumnBatch.scala | 21 +-- .../execution/columnar/ColumnTableScan.scala | 11 +- .../columnar/ExternalStoreUtils.scala | 4 +- .../execution/columnar/impl/ColumnDelta.scala | 9 +- .../columnar/impl/ColumnFormatEntry.scala | 1 - .../columnar/impl/ColumnFormatIterator.scala | 7 +- .../impl/JDBCSourceAsColumnarStore.scala | 24 ++- .../sql/execution/row/RowFormatScanRDD.scala | 4 +- .../sources/StoreDataSourceStrategy.scala | 3 +- .../spark/sql/row/JDBCMutableRelation.scala | 4 +- .../spark/sql/store/CompressionCodecId.scala | 7 +- .../apache/spark/sql/store/StoreUtils.scala | 3 +- .../snappydata/ColumnUpdateDeleteTests.scala | 35 +++- .../scala/io/snappydata/util/TestUtils.scala | 3 +- .../ColumnTableInternalValidationTest.scala | 16 ++ .../sql/streaming/SnappyStreamingSuite.scala | 4 +- store | 2 +- 27 files changed, 280 insertions(+), 168 deletions(-) diff --git a/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala b/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala index 9b7d5c587d..5e2aed3347 100644 --- a/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala +++ b/cluster/src/main/scala/org/apache/spark/memory/SnappyStorageEvictor.scala @@ -36,19 +36,19 @@ import org.apache.spark.sql.execution.columnar.impl.ColumnFormatRelation class SnappyStorageEvictor extends Logging { private def getAllRegionList(offHeap: Boolean, - hasOffHeap: Boolean): ArrayBuffer[LocalRegion] = { + hasOffHeap: Boolean, onlyRowBuffers: Boolean): ArrayBuffer[LocalRegion] = { val cache = GemFireCacheImpl.getExisting val allRegionList = new ArrayBuffer[LocalRegion]() val irm: InternalResourceManager = cache.getResourceManager for (listener <- irm.getResourceListeners( SnappyStorageEvictor.resourceType).asScala) listener match { case pr: PartitionedRegion => - if (includePartitionedRegion(pr, offHeap, hasOffHeap)) { + if (includePartitionedRegion(pr, offHeap, hasOffHeap, onlyRowBuffers)) { allRegionList ++= pr.getDataStore.getAllLocalBucketRegions.asScala } // no off-heap local regions yet in SnappyData case lr: LocalRegion => - if (!offHeap && includeLocalRegion(lr)) { + if (!offHeap && !onlyRowBuffers && includeLocalRegion(lr)) { allRegionList += lr } case _ => @@ -64,7 +64,8 @@ class SnappyStorageEvictor extends Logging { } @throws(classOf[Exception]) - def evictRegionData(bytesRequired: Long, offHeap: Boolean): Long = { + def evictRegionData(bytesRequired: Long, offHeap: Boolean, + onlyRowBuffers: Boolean = false): Long = { val cache = GemFireCacheImpl.getInstance() if (cache eq null) return 0L @@ -76,7 +77,7 @@ class SnappyStorageEvictor extends Logging { val stats = cache.getCachePerfStats stats.incEvictorJobsStarted() var totalBytesEvicted: Long = 0 - val regionSet = Random.shuffle(getAllRegionList(offHeap, hasOffHeap)) + val regionSet = Random.shuffle(getAllRegionList(offHeap, hasOffHeap, onlyRowBuffers)) val start = CachePerfStats.getStatTime try { while (regionSet.nonEmpty) { @@ -119,19 +120,26 @@ class SnappyStorageEvictor extends Logging { } stats.incEvictorJobsCompleted() } - totalBytesEvicted + // evict row-buffers as the last resort + if (!onlyRowBuffers && !offHeap && totalBytesEvicted < bytesRequired) { + totalBytesEvicted + evictRegionData(bytesRequired - totalBytesEvicted, + offHeap, onlyRowBuffers = true) + } else totalBytesEvicted } protected def includePartitionedRegion(region: PartitionedRegion, - offHeap: Boolean, hasOffHeap: Boolean): Boolean = { - val hasLRU = (region.getEvictionAttributes.getAlgorithm.isLRUHeap + offHeap: Boolean, hasOffHeap: Boolean, onlyRowBuffers: Boolean): Boolean = { + var hasLRU = (region.getEvictionAttributes.getAlgorithm.isLRUHeap && (region.getDataStore != null) - && !region.getAttributes.getEnableOffHeapMemory && !region.needsBatching()) + && !region.getAttributes.getEnableOffHeapMemory) + val isRowBuffer = region.needsBatching() + if (onlyRowBuffers) return hasLRU && isRowBuffer + + hasLRU &&= !isRowBuffer if (hasOffHeap) { // when off-heap is enabled then all column tables use off-heap - val regionPath = Misc.getFullTableNameFromRegionPath(region.getFullPath) - if (offHeap) hasLRU && ColumnFormatRelation.isColumnTable(regionPath) - else hasLRU && !ColumnFormatRelation.isColumnTable(regionPath) + hasLRU && offHeap == ColumnFormatRelation.isColumnTable( + Misc.getFullTableNameFromRegionPath(region.getFullPath)) } else { assert(!offHeap, "unexpected invocation for hasOffHeap=false and offHeap=true") diff --git a/cluster/src/test/scala/io/snappydata/QueryTest.scala b/cluster/src/test/scala/io/snappydata/QueryTest.scala index 468d969a99..8f6205ad6e 100644 --- a/cluster/src/test/scala/io/snappydata/QueryTest.scala +++ b/cluster/src/test/scala/io/snappydata/QueryTest.scala @@ -93,11 +93,19 @@ class QueryTest extends SnappyFunSuite { val query = "select k, v from t1 inner join t2 where t1.id = t2.k order by k, v" val df = session.sql(query) - val result1 = df.collect().mkString(" ") + var result1 = df.collect().mkString(" ") val result2 = spark.sql(query).collect().mkString(" ") if (result1 != result2) { fail(s"Expected result: $result2\nGot: $result1") } + + // force run stats so that small batches have been merged repeatedly + SnappyEmbeddedTableStatsProviderService.publishColumnTableRowCountStats() + Thread.sleep(10000) + result1 = df.collect().mkString(" ") + if (result1 != result2) { + fail(s"Expected result: $result2\nGot: $result1") + } } /** @@ -317,7 +325,7 @@ class QueryTest extends SnappyFunSuite { snc.sql(s"create index APP.X_TEST_COL3 on APP.TEST (col3)") snc.sql(s"insert into TEST values ('one', 'vone', 'cone'), ('two', 'vtwo', 'ctwo')") val r = snc.sql(s"select count(*) from TEST").collect() - assert (1 === r.size) + assert (1 === r.length) assert (2 === r.head.get(0)) snc.sql(s"ALTER TABLE APP.TEST ADD COLUMN COL5 blob") } diff --git a/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala b/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala index 892e64b20d..261c5816d3 100644 --- a/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala +++ b/cluster/src/test/scala/org/apache/spark/memory/SnappyStorageEvictorSuite.scala @@ -112,7 +112,7 @@ class SnappyStorageEvictorSuite extends MemoryFunSuite { var rows = 0 try { breakable { - for (i <- 1 to 20) { + for (i <- 1 to 50) { val row = Row(i, i, i) snSession.insert("t1", row) rows += 1 diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala index 0aba3ca72a..821c3f460f 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala @@ -23,7 +23,7 @@ import io.snappydata.cluster.PreparedQueryRoutingSingleNodeSuite import org.apache.spark.SparkConf import org.apache.spark.memory.SnappyUnifiedMemoryManager -import org.apache.spark.sql.{Row, SnappySession} +import org.apache.spark.sql.SnappySession /** * Tests for updates/deletes on column table. @@ -45,9 +45,9 @@ class ColumnUpdateDeleteTest extends ColumnTablesTestBase { conf.setIfMissing("spark.master", "local[*]") .setAppName(getClass.getName) conf.set("snappydata.store.critical-heap-percentage", "95") - if (SnappySession.isEnterpriseEdition) { - conf.set("snappydata.store.memory-size", "1200m") - } + // if (SnappySession.isEnterpriseEdition) { SW: + // conf.set("snappydata.store.memory-size", "1200m") + // } conf.set("spark.memory.manager", classOf[SnappyUnifiedMemoryManager].getName) conf.set("spark.serializer", "org.apache.spark.serializer.PooledKryoSerializer") conf.set("spark.closure.serializer", "org.apache.spark.serializer.PooledKryoSerializer") diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index a3419ec81f..c3b635a099 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -29,19 +29,20 @@ import scala.concurrent.Future import scala.language.implicitConversions import scala.util.control.NonFatal -import com.gemstone.gemfire.cache.IsolationLevel +import com.gemstone.gemfire.CancelException import com.gemstone.gemfire.cache.execute.FunctionService +import com.gemstone.gemfire.cache.{IsolationLevel, LockTimeoutException} import com.gemstone.gemfire.i18n.LogWriterI18n import com.gemstone.gemfire.internal.SystemTimer import com.gemstone.gemfire.internal.cache._ -import com.gemstone.gemfire.{CancelException, SystemFailure} import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.distributed.GfxdListResultCollector.ListResultCollectorValue import com.pivotal.gemfirexd.internal.engine.distributed.{GfxdListResultCollector, GfxdMessage} +import com.pivotal.gemfirexd.internal.engine.locks.GfxdLockSet import com.pivotal.gemfirexd.internal.engine.sql.execute.MemberStatisticsMessage import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer import com.pivotal.gemfirexd.internal.engine.ui._ -import io.snappydata.collection.ObjectObjectHashMap +import io.snappydata.collection.{ObjectObjectHashMap, OpenHashSet} import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.InternalRow @@ -269,23 +270,6 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService } } - private def handleException(t: Throwable): Unit = t match { - case e: Error if SystemFailure.isJVMFailureError(e) => - SystemFailure.initiateFailure(e) - // If this ever returns, rethrow the error. We're poisoned - // now, so don't let this thread continue. - throw e - case _ => - // Whenever you catch Error or Throwable, you must also - // check for fatal JVM error (see above). However, there is - // _still_ a possibility that you are dealing with a cascading - // error condition, so you also need to check to see if the JVM - // is still usable: - SystemFailure.checkFailure() - logWarning(t.getMessage, t) - throw t - } - private def handleTransaction(cache: GemFireCacheImpl, tx: TXStateProxy, context: TXManagerImpl.TXContext, success: Boolean): Unit = { if (tx ne null) { @@ -302,29 +286,17 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService } } - private def withExceptionHandling(f: => Unit, doFinally: () => Unit = null): Unit = { - try { - f - } catch { - case t: Throwable => handleException(t) - } finally { - if (doFinally ne null) doFinally() - } - } - def publishColumnTableRowCountStats(): Unit = { - def asSerializable[C](c: C) = c.asInstanceOf[C with Serializable] - val cache = Misc.getGemFireCache - val regions = asSerializable(cache.getApplicationRegions.asScala) - for (region: LocalRegion <- regions) { - if (region.getDataPolicy.withPartitioning()) { + val regions = cache.getApplicationRegions.iterator() + while (regions.hasNext) { + val region = regions.next() + val container = region.getUserAttribute.asInstanceOf[GemFireContainer] + if ((container ne null) && region.getDataPolicy.withPartitioning()) { val pr = region.asInstanceOf[PartitionedRegion] - val container = pr.getUserAttribute.asInstanceOf[GemFireContainer] - val isColumnTable = container.isColumnStore - if (isColumnTable && pr.getLocalMaxMemory > 0) { - val metaData = container.fetchHiveMetaData(false) - val schema = metaData.schema.asInstanceOf[StructType] + val columnMeta = if (container.isColumnStore) container.fetchHiveMetaData(false) else null + if ((columnMeta ne null) && pr.getLocalMaxMemory > 0) { + val schema = columnMeta.schema.asInstanceOf[StructType] val numColumnsInTable = schema.length // Resetting PR numRows in cached batch as this will be calculated every time. var rowsInColumnBatch = 0L @@ -339,7 +311,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService createRemoteIterator, false /* forUpdate */ , false /* includeValues */) val maxDeltaRows = pr.getColumnMaxDeltaRows var smallBucketRegion: BucketRegion = null - val smallBatchBuckets = new mutable.ArrayBuffer[BucketRegion](2) + val smallBatchBuckets = new OpenHashSet[BucketRegion](2) // using direct region operations while (itr.hasNext) { val re = itr.next().asInstanceOf[RegionEntry] @@ -347,15 +319,14 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val key = re.getRawKey.asInstanceOf[ColumnFormatKey] val bucketRegion = itr.getHostedBucketRegion if (bucketRegion.getBucketAdvisor.isPrimary) { - val batchRowCount = key.getColumnBatchRowCount(bucketRegion, - re, numColumnsInTable) + val batchRowCount = key.getColumnBatchRowCount(bucketRegion, re, + numColumnsInTable) rowsInColumnBatch += batchRowCount // check if bucket has multiple small batches if (key.getColumnIndex == ColumnFormatEntry.STATROW_COL_INDEX && batchRowCount < maxDeltaRows) { - val br = itr.getHostedBucketRegion - if (br eq smallBucketRegion) smallBatchBuckets += br - else smallBucketRegion = br + if (bucketRegion eq smallBucketRegion) smallBatchBuckets.add(bucketRegion) + else smallBucketRegion = bucketRegion } } re._getValue() match { @@ -366,14 +337,14 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService } itr.close() // submit a task to merge small batches if required - if (smallBatchBuckets.nonEmpty) { - mergeSmallColumnBatches(pr, container, metaData, smallBatchBuckets) + if (smallBatchBuckets.size() > 0) { + mergeSmallColumnBatches(pr, container, columnMeta, smallBatchBuckets.asScala) } } val stats = pr.getPrStats stats.setPRNumRowsInColumnBatches(rowsInColumnBatch) stats.setOffHeapSizeInBytes(offHeapSize) - } else if (!isColumnTable && pr.getLocalMaxMemory > 0 && container.isRowBuffer) { + } else if (container.isRowBuffer && pr.getLocalMaxMemory > 0) { rolloverTasks.computeIfAbsent(pr, rolloverRowBuffersTask) } } @@ -392,28 +363,51 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService * then roll it over into the column table */ private val rolloverRowBuffersTask = new JFunction[PartitionedRegion, Future[Unit]] { + + private def testBucket(br: BucketRegion, maxDeltaRows: Int, minModTime: Long): Boolean = { + val bucketSize = br.getRegionSize + bucketSize >= maxDeltaRows || (br.getLastModifiedTime <= minModTime && + bucketSize >= minSizeForRollover(br.getPartitionedRegion)) + } + override def apply(pr: PartitionedRegion): Future[Unit] = { val localPrimaries = pr.getDataStore.getAllLocalPrimaryBucketRegions if ((localPrimaries ne null) && localPrimaries.size() > 0) { - val doRollover = new Predicate[BucketRegion] { - private val minModTime = pr.getCache.cacheTimeMillis() - delayMillis - - override def test(br: BucketRegion): Boolean = { - br.getLastModifiedTime <= minModTime && br.getRegionSize >= minSizeForRollover(pr) + val maxDeltaRows = try { + pr.getColumnMaxDeltaRows + } catch { + case NonFatal(_) => return null + } + val minModTime = pr.getCache.cacheTimeMillis() - delayMillis + // minimize object creation in usual case with explicit iteration (rather than asScala) + var rolloverBuckets: OpenHashSet[BucketRegion] = null + val iter = localPrimaries.iterator() + while (iter.hasNext) { + val br = iter.next() + if (testBucket(br, maxDeltaRows, minModTime) && !br.columnBatchFlushLock.isWriteLocked) { + if (rolloverBuckets eq null) rolloverBuckets = new OpenHashSet[BucketRegion]() + rolloverBuckets.add(br) } } - val rolloverBuckets = localPrimaries.asScala.filter( - br => doRollover.test(br) && !br.columnBatchFlushLock.isWriteLocked) // enqueue a job to roll over required row buffers into column table // (each bucket will perform a last minute check before rollover inside lock) - if (rolloverBuckets.nonEmpty) { + if ((rolloverBuckets ne null) && rolloverBuckets.size() > 0) { implicit val executionContext = Utils.executionContext(pr.getGemFireCache) Future { + var locked = false try { - rolloverBuckets.foreach(bucket => withExceptionHandling( - bucket.createAndInsertColumnBatch(null, true, doRollover))) + locked = pr.lockForMaintenance(true, GfxdLockSet.MAX_LOCKWAIT_VAL) + if (locked) { + val doRollover = new Predicate[BucketRegion] { + override def test(br: BucketRegion): Boolean = + testBucket(br, maxDeltaRows, minModTime) + } + rolloverBuckets.asScala.foreach(bucket => Utils.withExceptionHandling( + bucket.createAndInsertColumnBatch(null, true, doRollover))) + } } finally { rolloverTasks.remove(pr) + if (locked) pr.unlockForMaintenance(true) } } } else null @@ -429,13 +423,24 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService * [[rolloverRowBuffersTask]] or a forced flush of even smaller size for sample tables. */ private def mergeSmallColumnBatches(pr: PartitionedRegion, container: GemFireContainer, - metaData: ExternalTableMetaData, smallBatches: mutable.ArrayBuffer[BucketRegion]): Unit = { + metaData: ExternalTableMetaData, smallBatchBuckets: mutable.Set[BucketRegion]): Unit = { mergeTasks.computeIfAbsent(pr, new JFunction[PartitionedRegion, Future[Unit]] { override def apply(pr: PartitionedRegion): Future[Unit] = { - logInfo(s"Found small batches in ${pr.getName}: ${smallBatches.map(_.getId)}") val cache = pr.getGemFireCache + val rowBuffer = GemFireContainer.getRowBufferTableName(container.getQualifiedTableName) + val rowBufferRegion = Misc.getRegionForTable(rowBuffer, true) + .asInstanceOf[PartitionedRegion] + var locked = false + + def releaseMaintenanceLock(): Unit = { + if (locked) { + rowBufferRegion.unlockForMaintenance(true) + locked = false + } + } + implicit val executionContext = Utils.executionContext(cache) - Future(withExceptionHandling({ + Future(Utils.withExceptionHandling({ val tableName = container.getQualifiedTableName val schema = metaData.schema.asInstanceOf[StructType] val maxDeltaRows = pr.getColumnMaxDeltaRows @@ -446,10 +451,10 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService otherRDDs = Nil, numBuckets = -1, partitionColumns = Nil, partitionColumnAliases = Nil, baseRelation = null, schema, allFilters = Nil, schemaAttrs, caseSensitive = true) - // reduce min delta row size to avoid going through rolloverRowBuffers again + // zero delta row size to avoid going through rolloverRowBuffers again val insertPlan = ColumnInsertExec(tableScan, Nil, Nil, numBuckets = -1, isPartitioned = false, None, - (pr.getColumnBatchSize, minSizeForRollover(pr), metaData.compressionCodec), + (pr.getColumnBatchSize, 0, metaData.compressionCodec), tableName, onExecutor = true, schema, metaData.externalStore.asInstanceOf[ExternalStore], useMemberVariables = false) // now generate the code with the help of WholeStageCodegenExec @@ -470,15 +475,25 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService var success = false var tx: TXStateProxy = null var context: TXManagerImpl.TXContext = null + logInfo(s"Found small batches in ${pr.getName}: " + + smallBatchBuckets.map(_.getId).mkString(", ")) // for each bucket, create an iterator to scan and insert the result batches; // a separate iterator is required because one ColumnInsertExec assumes a single batchId - for (br <- smallBatches) try { + for (br <- smallBatchBuckets) try { success = false - // start a new transaction for each bucket tx = null + // lock the row buffer region for maintenance operations + Thread.`yield`() // prefer any other foreground operations + locked = rowBufferRegion.lockForMaintenance(true, GfxdLockSet.MAX_LOCKWAIT_VAL) + if (!locked) { + throw new LockTimeoutException( + s"Failed to lock ${rowBufferRegion.getFullPath} for maintenance merge operation") + } + // start a new transaction for each bucket tx = if (cache.snapshotEnabled) { context = TXManagerImpl.getOrCreateTXContext() - cache.getCacheTransactionManager.beginTX(context, IsolationLevel.SNAPSHOT, null, null) + cache.getCacheTransactionManager.beginTX(context, + IsolationLevel.SNAPSHOT, null, null) } else null // find the committed entries with small batches under the transaction val bucketId = br.getId @@ -490,7 +505,8 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val re = itr.next().asInstanceOf[RegionEntry] if (!re.isDestroyedOrRemoved) { val key = re.getRawKey.asInstanceOf[ColumnFormatKey] - val batchRowCount = key.getColumnBatchRowCount(itr, re, schema.length) + val batchRowCount = key.getColumnBatchRowCount(itr.getHostedBucketRegion, + re, schema.length) // check if bucket has multiple small batches if (key.getColumnIndex == ColumnFormatEntry.STATROW_COL_INDEX && batchRowCount < maxDeltaRows) { @@ -521,11 +537,16 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService } success = true } catch { - case t: Throwable => handleException(t) + case le: LockTimeoutException => logWarning(le.getMessage) + case t: Throwable => Utils.logAndThrowException(t) } finally { handleTransaction(cache, tx, context, success) + releaseMaintenanceLock() } - }, () => mergeTasks.remove(pr))) + }, () => { + mergeTasks.remove(pr) + releaseMaintenanceLock() + })) } }) } diff --git a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala index 3aa108fac5..70bb62e4ba 100644 --- a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala +++ b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala @@ -542,7 +542,7 @@ object CachedDataFrame data.arrayOffset() + data.position(), data.remaining()) } - @transient private[sql] val nextExecutionIdMethod = { + @transient private[sql] lazy val nextExecutionIdMethod = { val m = SQLExecution.getClass.getDeclaredMethod("nextExecutionId") m.setAccessible(true) m diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index 8d8f3195ca..952c73488b 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -31,6 +31,7 @@ import scala.util.control.NonFatal import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.gemstone.gemfire.SystemFailure import com.gemstone.gemfire.internal.cache.GemFireCacheImpl import com.gemstone.gemfire.internal.shared.BufferAllocator import com.gemstone.gemfire.internal.shared.unsafe.UnsafeHolder @@ -65,7 +66,7 @@ import org.apache.spark.util.AccumulatorV2 import org.apache.spark.util.collection.BitSet import org.apache.spark.util.io.ChunkedByteBuffer -object Utils { +object Utils extends Logging { final val WEIGHTAGE_COLUMN_NAME = "SNAPPY_SAMPLER_WEIGHTAGE" final val SKIP_ANALYSIS_PREFIX = "SAMPLE_" @@ -88,6 +89,33 @@ object Utils { cause: Option[Throwable] = None): AnalysisException = new AnalysisException(msg, None, None, None, cause) + def withExceptionHandling(f: => Unit, doFinally: () => Unit = null): Unit = { + try { + f + } catch { + case t: Throwable => logAndThrowException(t) + } finally { + if (doFinally ne null) doFinally() + } + } + + def logAndThrowException(t: Throwable): Unit = t match { + case e: Error if SystemFailure.isJVMFailureError(e) => + SystemFailure.initiateFailure(e) + // If this ever returns, rethrow the error. We're poisoned + // now, so don't let this thread continue. + throw e + case _ => + // Whenever you catch Error or Throwable, you must also + // check for fatal JVM error (see above). However, there is + // _still_ a possibility that you are dealing with a cascading + // error condition, so you also need to check to see if the JVM + // is still usable: + SystemFailure.checkFailure() + logWarning(t.getMessage, t) + throw t + } + def columnIndex(col: String, cols: Array[String], module: String): Int = { val colT = toUpperCase(col.trim) cols.indices.collectFirst { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala index 32041ed447..226f68d732 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer -import com.gemstone.gemfire.internal.cache.LocalRegion +import com.gemstone.gemfire.internal.cache.{LocalRegion, PartitionedRegion} +import com.pivotal.gemfirexd.internal.engine.Misc +import com.pivotal.gemfirexd.internal.engine.locks.GfxdLockSet import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD @@ -30,7 +32,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, IndexColumnFormatRelation} -import org.apache.spark.sql.execution.columnar.{ColumnTableScan, ConnectionType} +import org.apache.spark.sql.execution.columnar.{ColumnDeleteExec, ColumnExec, ColumnTableScan, ColumnUpdateExec, ConnectionType} import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetricInfo, SQLMetrics} import org.apache.spark.sql.execution.row.{RowFormatRelation, RowTableScan} @@ -231,6 +233,21 @@ case class ExecutePlan(child: SparkPlan, preAction: () => Unit = () => ()) } protected[sql] lazy val sideEffectResult: Array[InternalRow] = { + SnappySession.getExecutedPlan(child)._1 match { + case c@(_: ColumnUpdateExec | _: ColumnDeleteExec) => + val pr = Misc.getRegionForTable(c.asInstanceOf[ColumnExec].resolvedName, true) + .asInstanceOf[PartitionedRegion] + val locked = pr.lockForMaintenance(false, GfxdLockSet.MAX_LOCKWAIT_VAL) + try { + getSideEffectResult + } finally { + if (locked) pr.unlockForMaintenance(false) + } + case _ => getSideEffectResult + } + } + + private def getSideEffectResult: Array[InternalRow] = { val session = sqlContext.sparkSession.asInstanceOf[SnappySession] val sc = session.sparkContext val key = session.currentKey @@ -261,7 +278,6 @@ case class ExecutePlan(child: SparkPlan, preAction: () => Unit = () => ()) (collectRDD(sc, rdd), shuffleIds) } if (shuffleIds.nonEmpty) { - logInfo(s"SW:0: got shuffleIds=$shuffleIds") sc.cleaner match { case Some(c) => shuffleIds.foreach(c.doCleanupShuffle(_, blocking = false)) case None => diff --git a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala index 05f55a0fd2..0807272db5 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/TableExec.scala @@ -176,7 +176,7 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor { /** * An iterator that will update provided metrics (those supported by an implementation). */ -trait IteratorWithMetrics[A] extends Iterator[A] { +abstract class IteratorWithMetrics[A] extends Iterator[A] { /** * Set a metric to be updated during iteration. diff --git a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index ac6ba80fd7..2556e3097d 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala @@ -114,15 +114,6 @@ case class SnappyHashAggregateExec( .map(_.toAttribute)) ++ AttributeSet(aggregateBufferAttributes) - private def getAliases(expressions: Seq[Expression], - existing: Seq[Seq[Attribute]]): Seq[Seq[Attribute]] = { - expressions.zipWithIndex.map { case (e, i) => - resultExpressions.collect { - case a@Alias(c, _) if c.semanticEquals(e) => a.toAttribute - } ++ (if (existing.isEmpty) Nil else existing(i)) - } - } - override def outputPartitioning: Partitioning = { child.outputPartitioning } @@ -456,11 +447,11 @@ case class SnappyHashAggregateExec( BindReferences.bindReference(e, inputAttrs).genCode(ctx) } s""" - $evaluateKeyVars - $evaluateBufferVars - $evaluateAggResults - ${consume(ctx, resultVars)} - """ + $evaluateKeyVars + $evaluateBufferVars + $evaluateAggResults + ${consume(ctx, resultVars)} + """ } else if (modes.contains(Partial) || modes.contains(PartialMerge)) { // Combined grouping keys and aggregate values in buffer diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala index 611d6f8258..b631eaee55 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnBatch.scala @@ -33,15 +33,15 @@ import com.pivotal.gemfirexd.internal.impl.jdbc.EmbedConnection import io.snappydata.collection.IntObjectHashMap import io.snappydata.thrift.common.BufferedBlob +import org.apache.spark.{Logging, TaskContext} import org.apache.spark.memory.MemoryManagerCallback -import org.apache.spark.sql.execution.SnappyMetrics import org.apache.spark.sql.execution.columnar.encoding.{ColumnDecoder, ColumnDeleteDecoder, ColumnEncoding, UpdatedColumnDecoder, UpdatedColumnDecoderBase} import org.apache.spark.sql.execution.columnar.impl._ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.row.PRValuesIterator +import org.apache.spark.sql.execution.{IteratorWithMetrics, SnappyMetrics} import org.apache.spark.sql.store.CompressionUtils import org.apache.spark.sql.types.StructField -import org.apache.spark.{Logging, TaskContext} case class ColumnBatch(numRows: Int, buffers: Array[ByteBuffer], statsData: Array[Byte], deltaIndexes: Array[Int]) @@ -49,7 +49,7 @@ case class ColumnBatch(numRows: Int, buffers: Array[ByteBuffer], abstract class ResultSetIterator[A](conn: Connection, stmt: Statement, rs: ResultSet, context: TaskContext, closeConnectionOnResultsClose: Boolean = true) - extends Iterator[A] with Logging { + extends IteratorWithMetrics[A] with Logging { protected[this] final var doMove = true @@ -85,6 +85,8 @@ abstract class ResultSetIterator[A](conn: Connection, protected def getCurrentValue: A + override def setMetric(name: String, metric: SQLMetric): Boolean = false + def close() { // if (!hasNextValue) return try { @@ -266,22 +268,11 @@ final class ColumnBatchIterator(region: LocalRegion, batch: ColumnBatch, if ((previousColumns ne null) && previousColumns.nonEmpty) { currentColumns = null val len = previousColumns.length - val checkDiskColumns = (diskBatchesFullMetric ne null) || (diskBatchesPartialMetric ne null) - var numDiskColumns = 0 var i = 0 while (i < len) { - val v = previousColumns(i) - if (checkDiskColumns && (v.getRegionContext eq null)) numDiskColumns += 1 - v.release() + previousColumns(i).release() i += 1 } - if (numDiskColumns > 0) { - if (numDiskColumns == len) { - if (diskBatchesFullMetric ne null) diskBatchesFullMetric.add(1) - } else if (diskBatchesPartialMetric ne null) { - diskBatchesPartialMetric.add(1) - } - } len } else 0 } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala index cb0b4786f6..8cff73075f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnTableScan.scala @@ -232,9 +232,9 @@ final case class ColumnTableScan( } else ("", "") val iteratorClass = "scala.collection.Iterator" + val iteratorWithMetricsClass = classOf[IteratorWithMetrics[_]].getName val colIteratorClass = if (embedded) classOf[ColumnBatchIterator].getName else classOf[ColumnBatchIteratorOnRS].getName - val iteratorWithMetricsClass = classOf[IteratorWithMetrics[_]].getName if (otherRDDs.isEmpty) { if (isForSampleReservoirAsRegion) { ctx.addMutableState(iteratorClass, rowInputSRR, @@ -353,12 +353,9 @@ final case class ColumnTableScan( val setColumnDiskMetricsSnippet = if (numBatchesDiskPartial eq null) "" else { s""" - if ($colInput instanceof $iteratorWithMetricsClass) { - $iteratorWithMetricsClass mIter = ($iteratorWithMetricsClass)$colInput; - mIter.setMetric("$NUM_BATCHES_DISK_PARTIAL", $numBatchesDiskPartial); - mIter.setMetric("$NUM_BATCHES_DISK_FULL", $numBatchesDiskFull); - mIter.setMetric("$NUM_BATCHES_REMOTE", $numBatchesRemote); - } + $colInput.setMetric("$NUM_BATCHES_DISK_PARTIAL", $numBatchesDiskPartial); + $colInput.setMetric("$NUM_BATCHES_DISK_FULL", $numBatchesDiskFull); + $colInput.setMetric("$NUM_BATCHES_REMOTE", $numBatchesRemote); """ } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala index 28f28f68b1..b8ad40ee1a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ExternalStoreUtils.scala @@ -718,11 +718,11 @@ object ExternalStoreUtils { def getExternalTableMetaData(qualifiedTable: String): ExternalTableMetaData = { getExternalTableMetaData(qualifiedTable, - GemFireXDUtils.getGemFireContainer(qualifiedTable, true), checkColumnStore = false) + GemFireXDUtils.getGemFireContainer(qualifiedTable, true)) } def getExternalTableMetaData(qualifiedTable: String, container: GemFireContainer, - checkColumnStore: Boolean): ExternalTableMetaData = { + checkColumnStore: Boolean = false): ExternalTableMetaData = { container match { case null => throw new IllegalStateException(s"Table $qualifiedTable not found in containers") diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala index 6e63bb87ad..5965555f48 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala @@ -30,6 +30,7 @@ import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer import org.apache.spark.sql.catalyst.expressions.{Add, AttributeReference, BoundReference, GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.collection.Utils +import org.apache.spark.sql.execution.columnar.ExternalStoreUtils import org.apache.spark.sql.execution.columnar.encoding.{ColumnDeltaEncoder, ColumnEncoding, ColumnStatsSchema} import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType} @@ -85,12 +86,8 @@ final class ColumnDelta extends ColumnFormatValue with Delta { val newValue = getValueRetain(FetchRequest.DECOMPRESS) val newBuffer = newValue.getBuffer try { - val schema = region.getUserAttribute.asInstanceOf[GemFireContainer] - .fetchHiveMetaData(false) match { - case null => throw new IllegalStateException( - s"Table for region ${region.getFullPath} not found in hive metadata") - case m => m.schema.asInstanceOf[StructType] - } + val schema = ExternalStoreUtils.getExternalTableMetaData(region.getFullPath, + region.getUserAttribute.asInstanceOf[GemFireContainer]).schema.asInstanceOf[StructType] val columnIndex = key.asInstanceOf[ColumnFormatKey].columnIndex // TODO: SW: if old value itself is returned, then avoid any put at GemFire layer // (perhaps throw some exception that can be caught and ignored in virtualPut) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala index 72d481923a..7f738f416f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala @@ -26,7 +26,6 @@ import com.gemstone.gemfire.cache.{DiskAccessException, EntryDestroyedException, import com.gemstone.gemfire.internal.DSFIDFactory.GfxdDSFID import com.gemstone.gemfire.internal.cache._ import com.gemstone.gemfire.internal.cache.lru.Sizeable -import com.gemstone.gemfire.internal.cache.partitioned.PREntriesIterator import com.gemstone.gemfire.internal.cache.persistence.DiskRegionView import com.gemstone.gemfire.internal.cache.store.SerializedDiskBuffer import com.gemstone.gemfire.internal.shared._ diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala index 842bd6f077..b94ceba1d4 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatIterator.scala @@ -427,7 +427,8 @@ private final class DiskMultiColumnBatch(_region: LocalRegion, _readerId: Int, * should use the normal ColumnFormatIterator. */ final class ColumnFormatStatsIterator(bucketRegion: BucketRegion, - statsEntries: Iterator[RegionEntry], tx: TXStateInterface) extends ClusteredDiskIterator { + statsEntries: Iterator[RegionEntry], tx: TXStateInterface) + extends ClusteredDiskIterator with DiskRegionIterator { try { bucketRegion.checkReadiness() @@ -454,5 +455,9 @@ final class ColumnFormatStatsIterator(bucketRegion: BucketRegion, bucketRegion.get(key, null, false, true, false, null, tx, null, null, false, false) } + override def initDiskIterator(): Boolean = false + + override def setRegion(region: LocalRegion): Unit = {} + override def close(): Unit = currentKey = null } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index ef3c9d6343..faef7ff785 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -97,6 +97,9 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie override def storeColumnBatch(columnTableName: String, batch: ColumnBatch, partitionId: Int, batchId: Long, maxDeltaRows: Int, compressionCodecId: Int, conn: Option[Connection]): Unit = { + if (batch.deltaIndexes ne null) { + logInfo(s"SW:1: applying updates on $tableName") + } // check for task cancellation before further processing checkTaskCancellation() if (partitionId >= 0) { @@ -113,6 +116,9 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie case Some(bucket) => bucket.updateInProgressSize(-batchSize) } } + if (batch.deltaIndexes ne null) { + logInfo(s"SW:1: DONE applying updates on $tableName") + } } // begin should decide the connection which will be used by insert/commit/rollback @@ -230,6 +236,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie override def storeDelete(columnTableName: String, buffer: ByteBuffer, partitionId: Int, batchId: Long, compressionCodecId: Int, conn: Option[Connection]): Unit = { + logInfo(s"SW:1: applying deletes on $tableName") // check for task cancellation before further processing checkTaskCancellation() val value = new ColumnDeleteDelta(buffer, compressionCodecId, isCompressed = false) @@ -319,6 +326,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie } }(conn) } + logInfo(s"SW:1: DONE applying deletes on $tableName") } def closeConnection(c: Option[Connection]): Unit = { @@ -366,7 +374,7 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie } // add the stats row val key = new ColumnFormatKey(batchId, partitionId, statRowIndex) - if (maxDeltaRows > 0 && maxDeltaRows < region.getColumnMaxDeltaRows) { + if (maxDeltaRows >= 0 && maxDeltaRows < region.getColumnMaxDeltaRows) { // log at info level for the case of column batch merges logInfo(s"Putting batch of size = ${batch.numRows} into ${region.getName}: $key") } else { @@ -1020,21 +1028,21 @@ class SmartConnectorRowRDD(_session: SnappySession, class SnapshotConnectionListener(store: JDBCSourceAsColumnarStore, delayRollover: Boolean) extends TaskCompletionListener { - val connAndTxId: Array[_ <: Object] = store.beginTx(delayRollover) - var isSuccess = false + private val connAndTxId: Array[_ <: Object] = store.beginTx(delayRollover) + private var isSuccess = false override def onTaskCompletion(context: TaskContext): Unit = { val txId = connAndTxId(1).asInstanceOf[String] - val conn = connAndTxId(0).asInstanceOf[Connection] - if (connAndTxId(1) != null) { + val conn = Option(connAndTxId(0).asInstanceOf[Connection]) + if (connAndTxId(1) ne null) { if (success()) { - store.commitTx(txId, delayRollover, Some(conn)) + store.commitTx(txId, delayRollover, conn) } else { - store.rollbackTx(txId, Some(conn)) + store.rollbackTx(txId, conn) } } - store.closeConnection(Some(conn)) + store.closeConnection(conn) } def success(): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index 4b57786bcc..da9ef7e8d7 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -288,7 +288,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, // use iterator over CompactExecRows directly when no projection; // higher layer PartitionedPhysicalRDD will take care of conversion // or direct code generation as appropriate - val itr = if (isPartitioned && filterWhereClause.isEmpty) { + val itr: IteratorWithMetrics[_] = if (isPartitioned && filterWhereClause.isEmpty) { val container = GemFireXDUtils.getGemFireContainer(tableName, true) val bucketIds = thePart match { case p: MultiBucketExecutorPartition => p.buckets @@ -421,7 +421,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, final class ResultSetTraversal(conn: Connection, stmt: Statement, val rs: ResultSet, context: TaskContext, source: Option[IteratorWithMetrics[_]] = None) - extends ResultSetIterator[Void](conn, stmt, rs, context) with IteratorWithMetrics[Void] { + extends ResultSetIterator[Void](conn, stmt, rs, context) { lazy val defaultCal: GregorianCalendar = ClientSharedData.getDefaultCleanCalendar diff --git a/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala b/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala index a2975a47bb..d9633768de 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/sources/StoreDataSourceStrategy.scala @@ -72,7 +72,7 @@ private[sql] object StoreDataSourceStrategy extends Strategy { 0, Nil, (a, f) => t.buildUnsafeScan(a.map(_.name).toArray, f.toArray)) :: Nil - case LogicalRelation(_, _, _) => { + case LogicalRelation(_, _, _) => var foundParamLiteral = false val tp = plan.transformAllExpressions { case pl: ParamLiteral => @@ -86,7 +86,6 @@ private[sql] object StoreDataSourceStrategy extends Strategy { } else { Nil } - } case _ => Nil } case _ => Nil diff --git a/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala b/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala index fd55eb2263..c102c1b326 100644 --- a/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/row/JDBCMutableRelation.scala @@ -19,7 +19,9 @@ package org.apache.spark.sql.row import java.sql.Connection import scala.collection.mutable -import io.snappydata.{Constant, SnappyTableStatsProviderService} + +import io.snappydata.SnappyTableStatsProviderService + import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow diff --git a/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala b/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala index c252a99a4d..831d5379fb 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/CompressionCodecId.scala @@ -40,11 +40,10 @@ object CompressionCodecId extends Enumeration { /** * The case of codec > MAX_ID should ideally be error but due to backward compatibility * the stats row does not have any header to determine compression or not so can fail - * in rare cases if first integer is a negative value. However it should never be match - * with the IDs here because negative of codecId which is written are -1, -2, -3 resolve + * in rare cases if first integer is a negative value. However it should never match + * the IDs here because negative of codecId which is written are -1, -2, -3 resolve * to 0xfffffff... which should never happen since nullCount fields are non-nullable - * (for not updated columns we keep -1 in null count) - * in the UnsafeRow created, so bitset cannot have 'ff' kind of patterns. + * in the UnsafeRow created, so bitset cannot have 'ff' kind of patterns. */ def isCompressed(codec: Int): Boolean = codec > 0 && codec <= MAX_ID diff --git a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala index 2006236a07..ba2a06505b 100644 --- a/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala +++ b/core/src/main/scala/org/apache/spark/sql/store/StoreUtils.scala @@ -418,8 +418,7 @@ object StoreUtils { parameters.remove(PARTITIONER).foreach(v => sb.append(GEM_PARTITIONER).append('\'').append(v).append("' ")) - // no overflow for row buffer tables - val overflow = (isRowTable || isShadowTable) && parameters.get(OVERFLOW).forall(_.toBoolean) + val overflow = parameters.get(OVERFLOW).forall(_.toBoolean) val defaultEviction = if (overflow) s"$GEM_HEAPPERCENT $GEM_OVERFLOW" else EMPTY_STRING sb.append(parameters.remove(EVICTION_BY).map(v => { if (v.contains(LRUCOUNT) && isShadowTable) { diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index 292c9b0274..b9d6567b74 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -17,13 +17,14 @@ package io.snappydata -import java.util.concurrent.{CyclicBarrier, Executors} +import java.util.concurrent.{CyclicBarrier, Executors, TimeUnit} import scala.collection.concurrent.TrieMap import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContext, Future} import com.gemstone.gemfire.internal.cache.{GemFireCacheImpl, PartitionedRegion} +import com.pivotal.gemfirexd.TestUtil import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer import io.snappydata.SnappyFunSuite.checkAnswer @@ -442,6 +443,12 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { } def testConcurrentOps(session: SnappySession): Unit = { + // start network server + val serverHostPort = TestUtil.startNetServer() + // scalastyle:off println + println(s"Started network server on $serverHostPort") + // scalastyle:on println + // reduced size to ensure both column table and row buffer have data session.conf.set(Property.ColumnBatchSize.name, "10k") // session.conf.set(Property.ColumnMaxDeltaRows.name, "200") @@ -474,7 +481,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { DistributedTestBase.invokeInEveryVM(avoidRollover) avoidRollover.run() - for (_ <- 1 to 3) { + for (_ <- 1 to 10000) { testConcurrentOpsIter(session) session.sql("truncate table updateTable") @@ -512,8 +519,8 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { .write.insertInto("checkTable2") val exceptions = new TrieMap[Thread, Throwable] - val executionContext = ExecutionContext.fromExecutorService( - Executors.newFixedThreadPool(concurrency + 2)) + val executorService = Executors.newFixedThreadPool(concurrency + 2) + val executionContext = ExecutionContext.fromExecutorService(executorService) // concurrent updates to different rows but same batches val barrier = new CyclicBarrier(concurrency) @@ -543,16 +550,24 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { var res = session.sql( "select * from updateTable EXCEPT select * from checkTable1").collect() + if (res.length != 0) { + // scalastyle:off println + println("Failed in updates?") + // scalastyle:on println + Thread.sleep(1000000) + } assert(res.length === 0) // concurrent deletes tasks = Array.tabulate(concurrency)(i => Future { + var awaitDone = false try { val snappy = new SnappySession(session.sparkContext) var res = snappy.sql("select count(*) from updateTable").collect() assert(res(0).getLong(0) === numElements) barrier.await() + awaitDone = true res = snappy.sql( s"delete from updateTable where (id % $step) = ${step - i - 1}").collect() assert(res.map(_.getLong(0)).sum > 0) @@ -560,6 +575,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { case t: Throwable => logError(t.getMessage, t) exceptions += Thread.currentThread() -> t + if (!awaitDone) barrier.await() throw t } }(executionContext)) @@ -569,7 +585,18 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { res = session.sql( "select * from updateTable EXCEPT select * from checkTable2").collect() + if (res.length != 0) { + // scalastyle:off println + println("Failed in deletes?") + // scalastyle:on println + Thread.sleep(1000000) + } assert(res.length === 0) + + executorService.shutdown() + if (!executorService.awaitTermination(1, TimeUnit.SECONDS)) { + executorService.shutdownNow() + } } def testSNAP2124(session: SnappySession, checkPruning: Boolean): Unit = { diff --git a/core/src/test/scala/io/snappydata/util/TestUtils.scala b/core/src/test/scala/io/snappydata/util/TestUtils.scala index 7eade840d0..4831b1a178 100644 --- a/core/src/test/scala/io/snappydata/util/TestUtils.scala +++ b/core/src/test/scala/io/snappydata/util/TestUtils.scala @@ -62,7 +62,8 @@ object TestUtils { (table, path) } allTablesWithRegions.filter { case (table, path) => - if (streams.exists(_.toString() == table)) { + if (path.startsWith("/SYS")) false + else if (streams.exists(_.toString() == table)) { false } else if (hasColocatedChildren(path, allRegions)) { parents += table diff --git a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala index e708cabe10..62ba69664f 100644 --- a/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala +++ b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.store import scala.util.Try +import com.gemstone.gemfire.cache.EvictionAlgorithm import com.gemstone.gemfire.internal.cache.PartitionedRegion import com.pivotal.gemfirexd.internal.engine.Misc import io.snappydata.{Property, SnappyFunSuite} @@ -70,6 +71,21 @@ class ColumnTableInternalValidationTest extends SnappyFunSuite logInfo("Success") } + test("test eviction for row buffer table") { + snc.sql(s"CREATE TABLE $tableName(Key1 INT ,Value STRING)" + + "USING column " + + "options " + + "(BUCKETS '200'," + + "REDUNDANCY '1'," + + "EVICTION_BY 'LRUHEAPPERCENT')").collect() + val rowBufferTable = s"APP.${tableName.toUpperCase}" + val rgn = Misc.getRegionForTable(rowBufferTable, true) + assert(rgn.getAttributes.getEvictionAttributes.getAlgorithm === EvictionAlgorithm.LRU_HEAP) + val colRgn = Misc.getRegionForTable( + ColumnFormatRelation.columnBatchTableName(rowBufferTable), true) + assert(colRgn.getAttributes.getEvictionAttributes.getAlgorithm === EvictionAlgorithm.LRU_HEAP) + } + test("test the shadow table with NOT NULL Column") { snc.sql(s"DROP TABLE IF EXISTS $tableName") snc.sql(s"CREATE TABLE $tableName(Key1 INT NOT NULL ,Value STRING) " + diff --git a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingSuite.scala b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingSuite.scala index be29d64412..620c10ba61 100644 --- a/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingSuite.scala +++ b/core/src/test/scala/org/apache/spark/sql/streaming/SnappyStreamingSuite.scala @@ -175,8 +175,8 @@ class SnappyStreamingSuite filtered.glom().foreachRDD(rdd => rdd.foreach(_.foreach(println))) val mapped = filtered.map(row => row.getString(0).toInt) mapped.foreachRDD(rdd => rdd.foreach(println)) - mapped.reduce(_ + _).foreachRDD(rdd => println(rdd.first())) - mapped.count().foreachRDD(rdd => println(rdd.first())) + mapped.reduce(_ + _).foreachRDD(_.foreach(println)) + mapped.count().foreachRDD(_.foreach(println)) // mapped.mapPartitions { _ => Seq.empty.toIterator } mapped.mapPartitions { x => Iterator(x.sum)} mapped.transform(rdd => rdd.map(_.toString)) diff --git a/store b/store index 53a1e34345..868f6a9a49 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 53a1e3434515b5b548d5709f185cb6c607920623 +Subproject commit 868f6a9a49cf0efd17abb8cf340c9b2e9df38980 From 0ea9eb994dd0faca651347e2fb9733d79783cb8c Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 29 Jun 2018 03:13:51 +0530 Subject: [PATCH 09/19] temp --- .../spark/sql/execution/columnar/impl/ColumnFormatEntry.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala index 7f738f416f..e2169c939f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala @@ -110,11 +110,10 @@ final class ColumnFormatKey(private[columnar] var uuid: Long, override def getColumnBatchRowCount(bucketRegion: BucketRegion, re: RegionEntry, numColumnsInTable: Int): Int = { - val currentBucketRegion = bucketRegion.getHostedBucketRegion if (columnIndex == ColumnFormatEntry.STATROW_COL_INDEX || columnIndex == ColumnFormatEntry.DELTA_STATROW_COL_INDEX || columnIndex == ColumnFormatEntry.DELETE_MASK_COL_INDEX) { - val statsOrDeleteVal = re.getValue(currentBucketRegion) + val statsOrDeleteVal = re.getValue(bucketRegion) if (statsOrDeleteVal ne null) { val statsOrDelete = statsOrDeleteVal.asInstanceOf[ColumnFormatValue] .getValueRetain(FetchRequest.DECOMPRESS) From c824cd6d2713614dea3b831bb1b089e7e46c1798 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Tue, 10 Jul 2018 14:12:26 +0530 Subject: [PATCH 10/19] minor formatting change --- .../execution/columnar/impl/StoreCallbacksImpl.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala index 0fc9bc2635..79e7b16118 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala @@ -16,11 +16,13 @@ */ package org.apache.spark.sql.execution.columnar.impl +import java.net.URLClassLoader import java.sql.SQLException import java.util.Collections import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer + import com.gemstone.gemfire.cache.{EntryDestroyedException, RegionDestroyedException} import com.gemstone.gemfire.internal.cache.lru.LRUEntry import com.gemstone.gemfire.internal.cache.persistence.query.CloseableIterator @@ -41,6 +43,7 @@ import com.pivotal.gemfirexd.internal.impl.jdbc.EmbedConnection import com.pivotal.gemfirexd.internal.shared.common.reference.SQLState import com.pivotal.gemfirexd.internal.snappy.LeadNodeSmartConnectorOpContext import io.snappydata.SnappyTableStatsProviderService + import org.apache.spark.memory.{MemoryManagerCallback, MemoryMode} import org.apache.spark.serializer.KryoSerializerPool import org.apache.spark.sql._ @@ -60,8 +63,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.{Logging, SparkContext} -import java.net.URLClassLoader - object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable { private val partitioner = new StoreHashFunction @@ -559,9 +560,9 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable } override def dropStorageMemory(objectName: String, ignoreBytes: Long): Unit = - // off-heap will be cleared via ManagedDirectBufferAllocator + // off-heap will be cleared via ManagedDirectBufferAllocator MemoryManagerCallback.memoryManager. - dropStorageMemoryForObject(objectName, MemoryMode.ON_HEAP, ignoreBytes) + dropStorageMemoryForObject(objectName, MemoryMode.ON_HEAP, ignoreBytes) override def waitForRuntimeManager(maxWaitMillis: Long): Unit = { val memoryManager = MemoryManagerCallback.memoryManager @@ -631,7 +632,7 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable cache.invalidateAll() } - override def getLeadClassLoader() : URLClassLoader = + override def getLeadClassLoader: URLClassLoader = ToolsCallbackInit.toolsCallback.getLeadClassLoader() } From bf26063ed5e3928021e490a83cd6892350dff913 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 2 Aug 2018 14:15:22 +0530 Subject: [PATCH 11/19] fixes --- .../sql/store/ColumnUpdateDeleteTest.scala | 6 +- .../SnappyTableStatsProviderService.scala | 52 +++----- .../impl/SmartConnectorRDDHelper.scala | 28 +++-- .../spark/sql/SmartConnectorHelper.scala | 7 +- .../org/apache/spark/sql/SnappySession.scala | 24 +++- .../apache/spark/sql/collection/Utils.scala | 14 ++- .../spark/sql/execution/ExistingPlans.scala | 21 +--- .../sql/execution/columnar/ColumnExec.scala | 7 +- .../execution/columnar/ColumnInsertExec.scala | 8 +- .../execution/columnar/impl/ColumnDelta.scala | 4 +- .../columnar/impl/ColumnFormatEncoder.scala | 4 +- .../columnar/impl/ColumnFormatEntry.scala | 3 +- .../columnar/impl/ColumnFormatRelation.scala | 8 +- .../impl/JDBCSourceAsColumnarStore.scala | 119 +++++++++--------- .../columnar/impl/StoreCallbacksImpl.scala | 12 +- .../sql/execution/row/RowFormatScanRDD.scala | 39 ++++-- .../sql/internal/ColumnTableBulkOps.scala | 2 +- store | 2 +- 18 files changed, 193 insertions(+), 167 deletions(-) diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala index 821c3f460f..226f3080d3 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala @@ -45,9 +45,9 @@ class ColumnUpdateDeleteTest extends ColumnTablesTestBase { conf.setIfMissing("spark.master", "local[*]") .setAppName(getClass.getName) conf.set("snappydata.store.critical-heap-percentage", "95") - // if (SnappySession.isEnterpriseEdition) { SW: - // conf.set("snappydata.store.memory-size", "1200m") - // } + if (SnappySession.isEnterpriseEdition) { + conf.set("snappydata.store.memory-size", "1200m") + } conf.set("spark.memory.manager", classOf[SnappyUnifiedMemoryManager].getName) conf.set("spark.serializer", "org.apache.spark.serializer.PooledKryoSerializer") conf.set("spark.closure.serializer", "org.apache.spark.serializer.PooledKryoSerializer") diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index d662a07589..c274bc2465 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -52,7 +52,6 @@ import org.apache.spark.sql.execution.columnar.{ColumnBatchIterator, ColumnInser import org.apache.spark.sql.execution.row.ResultSetTraversal import org.apache.spark.sql.execution.{BufferedRowIterator, WholeStageCodegenExec} import org.apache.spark.sql.store.CodeGeneration -import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{SnappyContext, ThinClientConnectorMode} /* @@ -300,8 +299,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val pr = region.asInstanceOf[PartitionedRegion] val columnMeta = if (container.isColumnStore) container.fetchHiveMetaData(false) else null if ((columnMeta ne null) && pr.getLocalMaxMemory > 0) { - val schema = columnMeta.schema.asInstanceOf[StructType] - val numColumnsInTable = schema.length + val numColumnsInTable = Utils.getTableSchema(columnMeta).length // Resetting PR numRows in cached batch as this will be calculated every time. var rowsInColumnBatch = 0L var offHeapSize = 0L @@ -388,7 +386,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val iter = localPrimaries.iterator() while (iter.hasNext) { val br = iter.next() - if (testBucket(br, maxDeltaRows, minModTime) && !br.columnBatchFlushLock.isWriteLocked) { + if (testBucket(br, maxDeltaRows, minModTime) && !br.isLockededForMaintenance) { if (rolloverBuckets eq null) rolloverBuckets = new OpenHashSet[BucketRegion]() rolloverBuckets.add(br) } @@ -398,20 +396,16 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService if ((rolloverBuckets ne null) && rolloverBuckets.size() > 0) { implicit val executionContext = Utils.executionContext(pr.getGemFireCache) Future { - var locked = false try { - locked = pr.lockForMaintenance(true, GfxdLockSet.MAX_LOCKWAIT_VAL) - if (locked) { - val doRollover = new Predicate[BucketRegion] { - override def test(br: BucketRegion): Boolean = - testBucket(br, maxDeltaRows, minModTime) - } - rolloverBuckets.asScala.foreach(bucket => Utils.withExceptionHandling( - bucket.createAndInsertColumnBatch(null, true, doRollover))) + val doRollover = new Predicate[BucketRegion] { + override def test(br: BucketRegion): Boolean = + testBucket(br, maxDeltaRows, minModTime) } + rolloverBuckets.asScala.foreach(bucket => Utils.withExceptionHandling( + bucket.createAndInsertColumnBatch(null, true, + GfxdLockSet.MAX_LOCKWAIT_VAL, doRollover))) } finally { rolloverTasks.remove(pr) - if (locked) pr.unlockForMaintenance(true) } } } else null @@ -431,22 +425,10 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService mergeTasks.computeIfAbsent(pr, new JFunction[PartitionedRegion, Future[Unit]] { override def apply(pr: PartitionedRegion): Future[Unit] = { val cache = pr.getGemFireCache - val rowBuffer = GemFireContainer.getRowBufferTableName(container.getQualifiedTableName) - val rowBufferRegion = Misc.getRegionForTable(rowBuffer, true) - .asInstanceOf[PartitionedRegion] - var locked = false - - def releaseMaintenanceLock(): Unit = { - if (locked) { - rowBufferRegion.unlockForMaintenance(true) - locked = false - } - } - implicit val executionContext = Utils.executionContext(cache) Future(Utils.withExceptionHandling({ val tableName = container.getQualifiedTableName - val schema = metaData.schema.asInstanceOf[StructType] + val schema = Utils.getTableSchema(metaData) val maxDeltaRows = pr.getColumnMaxDeltaRows val compileKey = tableName.concat(".MERGE_SMALL_BATCHES") val gen = CodeGeneration.compileCode(compileKey, schema.fields, () => { @@ -476,7 +458,9 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService // full projection for the iterators val numColumns = schema.length val projection = (1 to numColumns).toArray + val lockOwner = Thread.currentThread() var success = false + var locked = false var tx: TXStateProxy = null var context: TXManagerImpl.TXContext = null logInfo(s"Found small batches in ${pr.getName}: " + @@ -485,13 +469,14 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService // a separate iterator is required because one ColumnInsertExec assumes a single batchId for (br <- smallBatchBuckets) try { success = false + locked = false tx = null - // lock the row buffer region for maintenance operations - Thread.`yield`() // prefer any other foreground operations - locked = rowBufferRegion.lockForMaintenance(true, GfxdLockSet.MAX_LOCKWAIT_VAL) + // lock the row buffer bucket for maintenance operations + Thread.`yield`() // prefer foreground operations + locked = br.lockForMaintenance(true, GfxdLockSet.MAX_LOCKWAIT_VAL, lockOwner) if (!locked) { throw new LockTimeoutException( - s"Failed to lock ${rowBufferRegion.getFullPath} for maintenance merge operation") + s"Failed to lock ${br.getFullPath} for maintenance merge operation") } // start a new transaction for each bucket tx = if (cache.snapshotEnabled) { @@ -545,11 +530,12 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService case t: Throwable => Utils.logAndThrowException(t) } finally { handleTransaction(cache, tx, context, success) - releaseMaintenanceLock() + if (locked) { + br.unlockAfterMaintenance(true, lockOwner) + } } }, () => { mergeTasks.remove(pr) - releaseMaintenanceLock() })) } }) diff --git a/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala b/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala index c63f445cee..721b33ff72 100644 --- a/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala +++ b/core/src/main/scala/io/snappydata/impl/SmartConnectorRDDHelper.scala @@ -165,6 +165,7 @@ object SmartConnectorRDDHelper { if (bucketToServerMappingStr != null) { // check if Spark executors are using IP addresses or host names val preferHost = preferHostName(session) + val preferPrimaries = session.preferPrimaries val arr: Array[String] = bucketToServerMappingStr.split(":") var orphanBuckets: ArrayBuffer[Int] = null val noOfBuckets = arr(0).toInt @@ -177,15 +178,24 @@ object SmartConnectorRDDHelper { val aBucketInfo: Array[String] = x.split(";") val bid: Int = aBucketInfo(0).toInt if (!(aBucketInfo(1) == "null")) { - // get (host,addr,port) - val hostAddressPort = returnHostPortFromServerString(aBucketInfo(1)) - val hostName = hostAddressPort._1 - val host = if (preferHost) hostName else hostAddressPort._2 - val netUrl = urlPrefix + hostName + "[" + hostAddressPort._3 + "]" + urlSuffix - val netUrls = new ArrayBuffer[(String, String)](1) - netUrls += host -> netUrl - allNetUrls(bid) = netUrls - availableNetUrls.putIfAbsent(host, netUrl) + val n = aBucketInfo.length + val netUrls = new ArrayBuffer[(String, String)](n - 1) + var i = 1 + while (i < n) { + // get (host,addr,port) + val server = aBucketInfo(i) + if (server != "null") { + val hostAddressPort = returnHostPortFromServerString(server) + val hostName = hostAddressPort._1 + val host = if (preferHost) hostName else hostAddressPort._2 + val netUrl = urlPrefix + hostName + "[" + hostAddressPort._3 + "]" + urlSuffix + availableNetUrls.putIfAbsent(host, netUrl) + netUrls += host -> netUrl + } + i += 1 + } + allNetUrls(bid) = + if (preferPrimaries || n == 1) netUrls else scala.util.Random.shuffle(netUrls) } else { // Save the bucket which does not have a neturl, // and later assign available ones to it. diff --git a/core/src/main/scala/org/apache/spark/sql/SmartConnectorHelper.scala b/core/src/main/scala/org/apache/spark/sql/SmartConnectorHelper.scala index 5262ddd3ea..71c779c09d 100644 --- a/core/src/main/scala/org/apache/spark/sql/SmartConnectorHelper.scala +++ b/core/src/main/scala/org/apache/spark/sql/SmartConnectorHelper.scala @@ -17,15 +17,18 @@ package org.apache.spark.sql import java.io._ -import java.net.{URI, URL} +import java.net.URL import java.nio.file.{Files, Paths} import java.sql.{CallableStatement, Connection, SQLException} +import scala.collection.mutable + import com.pivotal.gemfirexd.Attribute import com.pivotal.gemfirexd.internal.shared.common.reference.SQLState import io.snappydata.Constant import io.snappydata.impl.SmartConnectorRDDHelper import org.apache.hadoop.hive.ql.metadata.Table + import org.apache.spark.sql.catalyst.expressions.SortDirection import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} @@ -34,8 +37,6 @@ import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.MutableURLClassLoader import org.apache.spark.{Logging, Partition, SparkContext} -import scala.collection.mutable - class SmartConnectorHelper(snappySession: SnappySession) extends Logging { private lazy val clusterMode = SnappyContext.getClusterMode(snappySession.sparkContext) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index 6a5cef8e93..a892e86daa 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -246,7 +246,7 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { java.util.Collections.unmodifiableMap(queryHints) @transient - private val contextObjects = new ConcurrentHashMap[Any, Any](16, 0.7f, 1) + private[sql] val contextObjects = new ConcurrentHashMap[Any, Any](16, 0.7f, 1) @transient private[sql] var currentKey: CachedKey = _ @@ -315,8 +315,22 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { } } - private[sql] def preferPrimaries: Boolean = - Property.PreferPrimariesInQuery.get(sessionState.conf) + private[sql] def setMutablePlan(mutable: Boolean): Unit = { + if (mutable) { + // use a unique lock owner + val lockOwner = s"MUTABLE_OP_OWNER_$id.${System.nanoTime()}" + addContextObject[String](SnappySession.MUTABLE_PLAN_OWNER, lockOwner) + } else removeContextObject(SnappySession.MUTABLE_PLAN_OWNER) + } + + private[sql] def isMutablePlan: Boolean = + contextObjects.containsKey(SnappySession.MUTABLE_PLAN_OWNER) + + private[sql] def getMutablePlanOwner: String = + contextObjects.get(SnappySession.MUTABLE_PLAN_OWNER).asInstanceOf[String] + + def preferPrimaries: Boolean = + Property.PreferPrimariesInQuery.get(sessionState.conf) || isMutablePlan private[sql] def addFinallyCode(ctx: CodegenContext, code: String): Int = { val depth = getContextObject[Int](ctx, "D", "depth").getOrElse(0) + 1 @@ -1874,6 +1888,9 @@ object SnappySession extends Logging { private[sql] val ExecutionKey = "EXECUTION" private[sql] val CACHED_PUTINTO_UPDATE_PLAN = "cached_putinto_logical_plan" + // internal property to indicate update/delete/putInto execution and lock owner for the same + private[sql] val MUTABLE_PLAN_OWNER = "snappydata.internal.mutablePlanOwner" + private[sql] var tokenize: Boolean = _ lazy val isEnterpriseEdition: Boolean = { @@ -1967,6 +1984,7 @@ object SnappySession extends Logging { // TODO add caching for point updates/deletes; a bit of complication // because getPlan will have to do execution with all waits/cleanups // normally done in CachedDataFrame.collectWithHandler/withCallback + // also reference objects like "updateOwner" need to be refreshed in every execution /* val cachedRDD = plan match { case p: ExecutePlan => p.child.execute() diff --git a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala index 952c73488b..a7f53b0b06 100644 --- a/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala +++ b/core/src/main/scala/org/apache/spark/sql/collection/Utils.scala @@ -32,7 +32,7 @@ import scala.util.control.NonFatal import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Kryo, KryoSerializable} import com.gemstone.gemfire.SystemFailure -import com.gemstone.gemfire.internal.cache.GemFireCacheImpl +import com.gemstone.gemfire.internal.cache.{ExternalTableMetaData, GemFireCacheImpl} import com.gemstone.gemfire.internal.shared.BufferAllocator import com.gemstone.gemfire.internal.shared.unsafe.UnsafeHolder import com.pivotal.gemfirexd.internal.engine.jdbc.GemFireXDRuntimeException @@ -56,7 +56,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, analysis} import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, DriverWrapper} import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.hive.SnappyStoreHiveCatalog +import org.apache.spark.sql.hive.{ExternalTableType, SnappyStoreHiveCatalog} import org.apache.spark.sql.sources.CastLongTime import org.apache.spark.sql.types._ import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerId} @@ -555,6 +555,16 @@ object Utils extends Logging { def schemaAttributes(schema: StructType): Seq[AttributeReference] = schema.toAttributes + def getTableSchema(metadata: ExternalTableMetaData): StructType = { + // add weightage column for sample tables + val schema = metadata.schema.asInstanceOf[StructType] + if (metadata.tableType == ExternalTableType.Sample.name && + schema(schema.length - 1).name != Utils.WEIGHTAGE_COLUMN_NAME) { + schema.add(Utils.WEIGHTAGE_COLUMN_NAME, + LongType, nullable = false) + } else schema + } + def getFields(o: Any): Map[String, Any] = { val fieldsAsPairs = for (field <- o.getClass.getDeclaredFields) yield { field.setAccessible(true) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala index 226f68d732..e5cfdca084 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala @@ -18,9 +18,7 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer -import com.gemstone.gemfire.internal.cache.{LocalRegion, PartitionedRegion} -import com.pivotal.gemfirexd.internal.engine.Misc -import com.pivotal.gemfirexd.internal.engine.locks.GfxdLockSet +import com.gemstone.gemfire.internal.cache.LocalRegion import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD @@ -32,7 +30,7 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, IndexColumnFormatRelation} -import org.apache.spark.sql.execution.columnar.{ColumnDeleteExec, ColumnExec, ColumnTableScan, ColumnUpdateExec, ConnectionType} +import org.apache.spark.sql.execution.columnar.{ColumnTableScan, ConnectionType} import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetricInfo, SQLMetrics} import org.apache.spark.sql.execution.row.{RowFormatRelation, RowTableScan} @@ -233,21 +231,6 @@ case class ExecutePlan(child: SparkPlan, preAction: () => Unit = () => ()) } protected[sql] lazy val sideEffectResult: Array[InternalRow] = { - SnappySession.getExecutedPlan(child)._1 match { - case c@(_: ColumnUpdateExec | _: ColumnDeleteExec) => - val pr = Misc.getRegionForTable(c.asInstanceOf[ColumnExec].resolvedName, true) - .asInstanceOf[PartitionedRegion] - val locked = pr.lockForMaintenance(false, GfxdLockSet.MAX_LOCKWAIT_VAL) - try { - getSideEffectResult - } finally { - if (locked) pr.unlockForMaintenance(false) - } - case _ => getSideEffectResult - } - } - - private def getSideEffectResult: Array[InternalRow] = { val session = sqlContext.sparkSession.asInstanceOf[SnappySession] val sc = session.sparkContext val key = session.currentKey diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala index f8e0f3dc75..3b3b7ad013 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.columnar import java.sql.Connection +import org.apache.spark.sql.SnappySession import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.{JDBCSourceAsColumnarStore, SnapshotConnectionListener} @@ -39,7 +40,10 @@ trait ColumnExec extends RowExec { override protected def connectionCodes(ctx: CodegenContext): (String, String, String) = { val connectionClass = classOf[Connection].getName + val session = sqlContext.sparkSession.asInstanceOf[SnappySession] val externalStoreTerm = ctx.addReferenceObj("externalStore", externalStore) + val updateOwner = ctx.addReferenceObj("updateOwner", + session.getMutablePlanOwner, classOf[String].getName) val listenerClass = classOf[SnapshotConnectionListener].getName val storeClass = classOf[JDBCSourceAsColumnarStore].getName taskListener = ctx.freshName("taskListener") @@ -51,7 +55,8 @@ trait ColumnExec extends RowExec { val initCode = s""" - |$taskListener = new $listenerClass(($storeClass)$externalStoreTerm, $delayRollover); + |$taskListener = new $listenerClass(($storeClass)$externalStoreTerm, + | $delayRollover, $updateOwner); |$connTerm = $taskListener.getConn(); |if ($getContext() != null) { | $getContext().addTaskCompletionListener($taskListener); diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala index 170adcbfa0..9957f8e2ae 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala @@ -588,14 +588,14 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String], ctx.addNewFunction(commitSnapshotTx, s""" |private final void $commitSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.commitTx($txId, false, $conn); + | $externalStoreTerm.commitTx($txId, false, null, -1, $conn); |} """.stripMargin) rollbackSnapshotTx = ctx.freshName("rollbackSnapshotTx") ctx.addNewFunction(rollbackSnapshotTx, s""" |private final void $rollbackSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.rollbackTx($txId, $conn); + | $externalStoreTerm.rollbackTx($txId, null, -1, $conn); |} """.stripMargin) closeConnection = ctx.freshName("closeConnection") @@ -725,14 +725,14 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String], ctx.addNewFunction(commitSnapshotTx, s""" |private final void $commitSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.commitTx($txId, false, $conn); + | $externalStoreTerm.commitTx($txId, false, null, -1, $conn); |} """.stripMargin) rollbackSnapshotTx = ctx.freshName("rollbackSnapshotTx") ctx.addNewFunction(rollbackSnapshotTx, s""" |private final void $rollbackSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.rollbackTx($txId, $conn); + | $externalStoreTerm.rollbackTx($txId, null, -1, $conn); |} """.stripMargin) closeConnection = ctx.freshName("closeConnection") diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala index 5965555f48..12a5727847 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala @@ -86,8 +86,8 @@ final class ColumnDelta extends ColumnFormatValue with Delta { val newValue = getValueRetain(FetchRequest.DECOMPRESS) val newBuffer = newValue.getBuffer try { - val schema = ExternalStoreUtils.getExternalTableMetaData(region.getFullPath, - region.getUserAttribute.asInstanceOf[GemFireContainer]).schema.asInstanceOf[StructType] + val schema = Utils.getTableSchema(ExternalStoreUtils.getExternalTableMetaData( + region.getFullPath, region.getUserAttribute.asInstanceOf[GemFireContainer])) val columnIndex = key.asInstanceOf[ColumnFormatKey].columnIndex // TODO: SW: if old value itself is returned, then avoid any put at GemFire layer // (perhaps throw some exception that can be caught and ignored in virtualPut) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala index d8ed2a8df2..30c3676db4 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEncoder.scala @@ -30,9 +30,9 @@ import com.pivotal.gemfirexd.internal.impl.sql.execute.ValueRow import io.snappydata.thrift.common.BufferedBlob import io.snappydata.thrift.internal.ClientBlob +import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.encoding.ColumnDeleteDelta import org.apache.spark.sql.store.CompressionCodecId -import org.apache.spark.sql.types.StructType /** * A [[RowEncoder]] implementation for [[ColumnFormatValue]] and child classes. @@ -136,7 +136,7 @@ final class ColumnFormatEncoder extends RowEncoder { } if (deleteBatch) { val container = region.getUserAttribute.asInstanceOf[GemFireContainer] - val schema = container.fetchHiveMetaData(false).schema.asInstanceOf[StructType] + val schema = Utils.getTableSchema(container.fetchHiveMetaData(false)) ColumnDelta.deleteBatch(deleteKey, region, schema.length) } case _ => diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala index e2169c939f..d20560855a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatEntry.scala @@ -44,7 +44,6 @@ import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.encoding.{ColumnDeleteDelta, ColumnEncoding, ColumnStatsSchema} import org.apache.spark.sql.execution.columnar.impl.ColumnFormatEntry.alignedSize import org.apache.spark.sql.store.{CompressionCodecId, CompressionUtils} -import org.apache.spark.sql.types.StructType /** * Utility methods for column format storage keys and values. @@ -106,7 +105,7 @@ final class ColumnFormatKey(private[columnar] var uuid: Long, def this() = this(-1L, -1, -1) override def getNumColumnsInTable(columnTable: GemFireContainer): Int = - columnTable.fetchHiveMetaData(false).schema.asInstanceOf[StructType].length + Utils.getTableSchema(columnTable.fetchHiveMetaData(false)).length override def getColumnBatchRowCount(bucketRegion: BucketRegion, re: RegionEntry, numColumnsInTable: Int): Int = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala index 3e6a60cdd9..2746dde175 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala @@ -262,6 +262,9 @@ abstract class BaseColumnFormatRelation( partitioningColumns.map(Utils.toUpperCase) ++ ColumnDelta.mutableKeyNames } + protected def markMutablePlan(): Unit = + sqlContext.sparkSession.asInstanceOf[SnappySession].setMutablePlan(true) + /** * Get a spark plan to update rows in the relation. The result of SparkPlan * execution should be a count of number of updated rows. @@ -269,6 +272,7 @@ abstract class BaseColumnFormatRelation( override def getUpdatePlan(relation: LogicalRelation, child: SparkPlan, updateColumns: Seq[Attribute], updateExpressions: Seq[Expression], keyColumns: Seq[Attribute]): SparkPlan = { + markMutablePlan() ColumnUpdateExec(child, externalColumnTableName, partitionColumns, partitionExpressions(relation), numBuckets, isPartitioned, schema, externalStore, this, updateColumns, updateExpressions, keyColumns, connProperties, onExecutor = false) @@ -280,6 +284,7 @@ abstract class BaseColumnFormatRelation( */ override def getDeletePlan(relation: LogicalRelation, child: SparkPlan, keyColumns: Seq[Attribute]): SparkPlan = { + markMutablePlan() ColumnDeleteExec(child, externalColumnTableName, partitionColumns, partitionExpressions(relation), numBuckets, isPartitioned, schema, externalStore, this, keyColumns, connProperties, onExecutor = false) @@ -683,6 +688,7 @@ class ColumnFormatRelation( * The result of SparkPlan execution should be a count of number of rows put. */ override def getPutPlan(insertPlan: SparkPlan, updatePlan: SparkPlan): SparkPlan = { + markMutablePlan() ColumnPutIntoExec(insertPlan, updatePlan) } @@ -777,7 +783,7 @@ object ColumnFormatRelation extends Logging with StoreCallback { assert(indexEntry.dml.nonEmpty) val rowInsertStr = indexEntry.dml (CodeGeneration.getGeneratedIndexStatement(indexEntry.entityName, - indexEntry.schema.asInstanceOf[StructType], + Utils.getTableSchema(indexEntry), indexEntry.externalStore.asInstanceOf[ExternalStore].connProperties.dialect), connectedExternalStore.conn.prepareStatement(rowInsertStr)) } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index faef7ff785..1569237c48 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -36,7 +36,7 @@ import com.pivotal.gemfirexd.internal.iapi.services.context.ContextService import com.pivotal.gemfirexd.internal.impl.jdbc.{EmbedConnection, EmbedConnectionContext} import io.snappydata.impl.SmartConnectorRDDHelper import io.snappydata.thrift.StatementAttrs -import io.snappydata.thrift.internal.{ClientBlob, ClientPreparedStatement, ClientStatement} +import io.snappydata.thrift.internal.{ClientBlob, ClientStatement} import org.apache.spark.rdd.RDD import org.apache.spark.serializer.{ConnectionPropertiesSerializer, KryoSerializerPool, StructTypeSerializer} @@ -97,9 +97,6 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie override def storeColumnBatch(columnTableName: String, batch: ColumnBatch, partitionId: Int, batchId: Long, maxDeltaRows: Int, compressionCodecId: Int, conn: Option[Connection]): Unit = { - if (batch.deltaIndexes ne null) { - logInfo(s"SW:1: applying updates on $tableName") - } // check for task cancellation before further processing checkTaskCancellation() if (partitionId >= 0) { @@ -116,9 +113,6 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie case Some(bucket) => bucket.updateInProgressSize(-batchSize) } } - if (batch.deltaIndexes ne null) { - logInfo(s"SW:1: DONE applying updates on $tableName") - } } // begin should decide the connection which will be used by insert/commit/rollback @@ -169,22 +163,32 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie }(Some(conn)) } - def commitTx(txId: String, delayRollover: Boolean, conn: Option[Connection]): Unit = { + def commitTx(txId: String, delayRollover: Boolean, updateOwner: String, + partitionId: Int, conn: Option[Connection]): Unit = { tryExecute(tableName, closeOnSuccessOrFailure = false, onExecutor = true)(conn => { var success = false try { connectionType match { - case ConnectionType.Embedded => + case ConnectionType.Embedded => try { // if rollover was marked as delayed, then do the rollover before commit if (delayRollover) { GfxdSystemProcedures.flushLocalBuckets(tableName, false) } Misc.getGemFireCache.getCacheTransactionManager.commit() + } finally { + if (updateOwner ne null) { + GfxdSystemProcedures.unlockBucketAfterMaintenance( + tableName, updateOwner, partitionId) + } + } case _ => logDebug(s"Going to commit $txId the transaction on server conn is $conn") - val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)") + val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?,?,?,?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, if (delayRollover) tableName else "") + ps.setString(2, tableName) + ps.setBoolean(3, delayRollover) + ps.setString(4, if (updateOwner ne null) updateOwner else "") + ps.setInt(5, partitionId) try { ps.executeUpdate() logDebug(s"The txid being committed is $txId") @@ -209,19 +213,30 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie } - def rollbackTx(txId: String, conn: Option[Connection]): Unit = { + def rollbackTx(txId: String, updateOwner: String, partitionId: Int, + conn: Option[Connection]): Unit = { // noinspection RedundantDefaultArgument tryExecute(tableName, closeOnSuccessOrFailure = true, onExecutor = true) { (conn: Connection) => { connectionType match { case ConnectionType.Embedded => - Misc.getGemFireCache.getCacheTransactionManager.rollback() + try { + Misc.getGemFireCache.getCacheTransactionManager.rollback() + } finally { + if (updateOwner ne null) { + GfxdSystemProcedures.unlockBucketAfterMaintenance( + tableName, updateOwner, partitionId) + } + } case _ => logDebug(s"Going to rollback transaction $txId on server using $conn") var ps: PreparedStatement = null handleRollback(() => { - ps = conn.prepareStatement(s"call sys.ROLLBACK_SNAPSHOT_TXID(?)") + ps = conn.prepareStatement(s"call sys.ROLLBACK_SNAPSHOT_TXID(?,?,?,?)") ps.setString(1, if (txId ne null) txId else "") + ps.setString(2, tableName) + ps.setString(3, if (updateOwner ne null) updateOwner else "") + ps.setInt(4, partitionId) ps.executeUpdate() logDebug(s"The transaction ID being rolled back is $txId") }, () => { @@ -236,7 +251,6 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie override def storeDelete(columnTableName: String, buffer: ByteBuffer, partitionId: Int, batchId: Long, compressionCodecId: Int, conn: Option[Connection]): Unit = { - logInfo(s"SW:1: applying deletes on $tableName") // check for task cancellation before further processing checkTaskCancellation() val value = new ColumnDeleteDelta(buffer, compressionCodecId, isCompressed = false) @@ -326,7 +340,6 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie } }(conn) } - logInfo(s"SW:1: DONE applying deletes on $tableName") } def closeConnection(c: Option[Connection]): Unit = { @@ -799,9 +812,12 @@ final class SmartConnectorColumnRDD( logDebug(s"The txid going to be committed is $txId " + tableName) // if ((txId ne null) && !txId.equals("null")) { - val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)") + val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?,?,?,?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, if (delayRollover) tableName else "") + ps.setString(2, tableName) + ps.setBoolean(3, delayRollover) + ps.setString(4, "") + ps.setInt(5, -1) ps.executeUpdate() logDebug(s"The txid being committed is $txId") ps.close() @@ -902,9 +918,12 @@ class SmartConnectorRowRDD(_session: SnappySession, val txId = SmartConnectorRDDHelper.snapshotTxIdForRead.get logDebug(s"The txid going to be committed is $txId " + tableName) // if ((txId ne null) && !txId.equals("null")) { - val ps = conn.get.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)") + val ps = conn.get.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?,?,?,?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, if (delayRollover) tableName else "") + ps.setString(2, tableName) + ps.setBoolean(3, delayRollover) + ps.setString(3, "") + ps.setInt(4, -1) ps.executeUpdate() logDebug(s"The txid being committed is $txId") ps.close() @@ -933,31 +952,18 @@ class SmartConnectorRowRDD(_session: SnappySession, val bucketPartition = thePart.asInstanceOf[SmartExecutorBucketPartition] logDebug(s"Scanning row buffer for $tableName,partId=${bucketPartition.index}," + s" bucketId = ${bucketPartition.bucketId}") - val statement = conn.createStatement() - val thriftConn = statement match { - case clientStmt: ClientStatement => - val clientConn = clientStmt.getConnection - if (isPartitioned) { - clientConn.setCommonStatementAttributes(ClientStatement.setLocalExecutionBucketIds( - new StatementAttrs(), Collections.singleton(Int.box(bucketPartition.bucketId)), - tableName, true).setMetadataVersion(relDestroyVersion)) - } - clientConn - case _ => null - } - if (isPartitioned && (thriftConn eq null)) { - val ps = conn.prepareStatement( - s"call sys.SET_BUCKETS_FOR_LOCAL_EXECUTION(?, ?, $relDestroyVersion)") - ps.setString(1, tableName) - val bucketString = bucketPartition.bucketId.toString - ps.setString(2, bucketString) - ps.executeUpdate() - ps.close() + val statement = conn.createStatement().asInstanceOf[ClientStatement] + // get the underlying thrift connection (conn is pool wrapper) + val thriftConn = statement.getConnection + if (isPartitioned) { + thriftConn.setCommonStatementAttributes(ClientStatement.setLocalExecutionBucketIds( + new StatementAttrs(), Collections.singleton(Int.box(bucketPartition.bucketId)), + tableName, true).setMetadataVersion(relDestroyVersion).setLockOwner(updateOwner)) } val sqlText = s"SELECT $columnList FROM ${quotedName(tableName)}$filterWhereClause" val args = filterWhereArgs - val stmt = conn.prepareStatement(sqlText) + val stmt = thriftConn.prepareStatement(sqlText) if (args ne null) { ExternalStoreUtils.setStatementParameters(stmt, args) } @@ -967,15 +973,11 @@ class SmartConnectorRowRDD(_session: SnappySession, } val txId = SmartConnectorRDDHelper.snapshotTxIdForRead.get - if (thriftConn ne null) { - stmt.asInstanceOf[ClientPreparedStatement].setSnapshotTransactionId(txId) - } else if (txId != null) { - if (!txId.isEmpty) { - statement.execute( - s"call sys.USE_SNAPSHOT_TXID('$txId')") - } - } + stmt.setSnapshotTransactionId(txId) + // TODO: SW: change to use prepareAndExecute but need to fix + // the types in ClientPreparedStatement.paramsList which will not + // be available before-hand and need to be changed as per parameter values val rs = stmt.executeQuery() // get the txid which was used to take the snapshot. @@ -990,9 +992,7 @@ class SmartConnectorRowRDD(_session: SnappySession, SmartConnectorRDDHelper.snapshotTxIdForRead.set(txId) logDebug(s"The snapshot tx id is $txId and tablename is $tableName") } - if (thriftConn ne null) { - thriftConn.setCommonStatementAttributes(null) - } + thriftConn.setCommonStatementAttributes(null) logDebug(s"The previous snapshot tx id is $txId and tablename is $tableName") (conn, stmt, rs) } @@ -1027,20 +1027,17 @@ class SmartConnectorRowRDD(_session: SnappySession, } class SnapshotConnectionListener(store: JDBCSourceAsColumnarStore, - delayRollover: Boolean) extends TaskCompletionListener { + delayRollover: Boolean, updateOwner: String) extends TaskCompletionListener { private val connAndTxId: Array[_ <: Object] = store.beginTx(delayRollover) private var isSuccess = false override def onTaskCompletion(context: TaskContext): Unit = { - val txId = connAndTxId(1).asInstanceOf[String] val conn = Option(connAndTxId(0).asInstanceOf[Connection]) - if (connAndTxId(1) ne null) { - if (success()) { - store.commitTx(txId, delayRollover, conn) - } - else { - store.rollbackTx(txId, conn) - } + val txId = connAndTxId(1).asInstanceOf[String] + if (success()) { + store.commitTx(txId, delayRollover, updateOwner, context.partitionId(), conn) + } else { + store.rollbackTx(txId, updateOwner, context.partitionId(), conn) } store.closeConnection(conn) } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala index 79e7b16118..239e971555 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/StoreCallbacksImpl.scala @@ -56,7 +56,7 @@ import org.apache.spark.sql.collection.{ToolsCallbackInit, Utils} import org.apache.spark.sql.execution.ConnectionPool import org.apache.spark.sql.execution.columnar.encoding.ColumnStatsSchema import org.apache.spark.sql.execution.columnar.{ColumnBatchCreator, ColumnBatchIterator, ColumnTableScan, ExternalStore, ExternalStoreUtils} -import org.apache.spark.sql.hive.{ExternalTableType, SnappyStoreHiveCatalog} +import org.apache.spark.sql.hive.SnappyStoreHiveCatalog import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.store.{CodeGeneration, StoreHashFunction} import org.apache.spark.sql.types._ @@ -118,13 +118,7 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable } else Nil val tableName = container.getQualifiedTableName - // add weightage column for sample tables if required - var schema = catalogEntry.schema.asInstanceOf[StructType] - if (catalogEntry.tableType == ExternalTableType.Sample.name && - schema(schema.length - 1).name != Utils.WEIGHTAGE_COLUMN_NAME) { - schema = schema.add(Utils.WEIGHTAGE_COLUMN_NAME, - LongType, nullable = false) - } + val schema = Utils.getTableSchema(catalogEntry) val batchCreator = new ColumnBatchCreator(pr, ColumnFormatRelation.columnBatchTableName(tableName, None), schema, catalogEntry.externalStore.asInstanceOf[ExternalStore], @@ -199,7 +193,7 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable val lr = Misc.getRegionForTable(columnTable, true).asInstanceOf[LocalRegion] val metadata = ExternalStoreUtils.getExternalTableMetaData(columnTable, lr.getUserAttribute.asInstanceOf[GemFireContainer], checkColumnStore = true) - val schema = metadata.schema.asInstanceOf[StructType].toAttributes + val schema = Utils.getTableSchema(metadata).toAttributes val filterExprs = if (batchFilters ne null) { batchFilters.map(f => translateFilter(f, schema)) } else null diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index da9ef7e8d7..462c795c47 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -67,6 +67,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, protected var delayRollover: Boolean, protected var projection: Array[Int]) extends RDDKryo[Any](session.sparkContext, Nil) with KryoSerializable { + protected var updateOwner: String = if (session ne null) session.getMutablePlanOwner else null protected var filterWhereArgs: ArrayBuffer[Any] = _ /** * `filters`, but as a WHERE clause suitable for injection into a SQL query. @@ -199,7 +200,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, if (isPartitioned) { val ps = conn.prepareStatement( - "call sys.SET_BUCKETS_FOR_LOCAL_EXECUTION(?, ?, ?)") + "call sys.SET_BUCKETS_FOR_LOCAL_EXECUTION_EX(?, ?, ?, ?)") try { ps.setString(1, tableName) val bucketString = thePart match { @@ -208,6 +209,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, } ps.setString(2, bucketString) ps.setInt(3, -1) + ps.setString(4, updateOwner) ps.executeUpdate() } finally { ps.close() @@ -272,6 +274,11 @@ class RowFormatScanRDD(@transient val session: SnappySession, } itr } else { + val container = GemFireXDUtils.getGemFireContainer(tableName, true) + val bucketIds = thePart match { + case p: MultiBucketExecutorPartition => p.buckets + case _ => java.util.Collections.singleton(Int.box(thePart.index)) + } // explicitly check authorization for the case of column table scan // !pushProjections && useResultSet means a column table if (useResultSet) { @@ -281,20 +288,27 @@ class RowFormatScanRDD(@transient val session: SnappySession, val txManagerImpl = GemFireCacheImpl.getExisting.getCacheTransactionManager var tx = txManagerImpl.getTXState val startTX = tx eq null - if (startTX) { - tx = txManagerImpl.beginTX(TXManagerImpl.getOrCreateTXContext, - IsolationLevel.SNAPSHOT, null, null) + // acquire bucket maintenance read lock if required before snapshot gets acquired + // and register to be released with transaction commit + val br = BucketRegion.lockPrimaryForMaintenance(false, updateOwner, + container.getRegion.asInstanceOf[PartitionedRegion], bucketIds) + var success = false + try { + if (startTX) { + tx = txManagerImpl.beginTX(TXManagerImpl.getOrCreateTXContext, + IsolationLevel.SNAPSHOT, null, null) + } + // register the locked region in TX for lock release in case of rollback + success = tx.getProxy.registerLockedBucketRegion(br) + } finally { + if ((br ne null) && !success) { + br.unlockAfterMaintenance(false, updateOwner) + } } // use iterator over CompactExecRows directly when no projection; // higher layer PartitionedPhysicalRDD will take care of conversion // or direct code generation as appropriate val itr: IteratorWithMetrics[_] = if (isPartitioned && filterWhereClause.isEmpty) { - val container = GemFireXDUtils.getGemFireContainer(tableName, true) - val bucketIds = thePart match { - case p: MultiBucketExecutorPartition => p.buckets - case _ => java.util.Collections.singleton(Int.box(thePart.index)) - } - val txId = if (tx ne null) tx.getTransactionId else null // always fault-in for row buffers val itr = new CompactExecRowIteratorOnScan(container, bucketIds, txId, @@ -373,6 +387,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, output.writeVarInt(projection.length, true) output.writeInts(projection, true) } + output.writeString(updateOwner) // need connection properties only if computing ResultSet if (pushProjections || useResultSet || !isPartitioned || len > 0) { ConnectionPropertiesSerializer.write(kryo, output, connProperties) @@ -406,6 +421,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, val numProjections = input.readVarInt(true) projection = input.readInts(numProjections, true) } + updateOwner = input.readString() // read connection properties only if computing ResultSet if (pushProjections || useResultSet || !isPartitioned || numFilters > 0) { connProperties = ConnectionPropertiesSerializer.read(kryo, input) @@ -445,7 +461,8 @@ final class CompactExecRowIteratorOnRS(conn: Connection, } abstract class PRValuesIterator[T](container: GemFireContainer, - region: LocalRegion, bucketIds: java.util.Set[Integer]) extends IteratorWithMetrics[T] { + region: LocalRegion, bucketIds: java.util.Set[Integer]) + extends IteratorWithMetrics[T] { protected type PRIterator = PartitionedRegion#PRLocalScanIterator diff --git a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala index 48c7273842..8138ee6133 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala @@ -169,7 +169,7 @@ object ColumnTableBulkOps { val deletePlan = Delete(table, exists, Nil) val deleteDs = new Dataset(sparkSession, deletePlan, RowEncoder(deletePlan.schema)) transFormedPlan = deleteDs.queryExecution.analyzed.asInstanceOf[Delete] - case lr@LogicalRelation(mutable: MutableRelation, _, _) => + case LogicalRelation(mutable: MutableRelation, _, _) => val ks = mutable.getKeyColumns if (ks.isEmpty) { throw new AnalysisException( diff --git a/store b/store index 3b11999b46..6f9682c910 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 3b11999b4605553da516579277348c569b017e9c +Subproject commit 6f9682c9109dde942ef44d22580cbd5e1b968777 From f423108cbf3064d01d6d2eb802dabf1419416f6b Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Thu, 2 Aug 2018 16:46:16 +0530 Subject: [PATCH 12/19] fix a ClassCast --- .../spark/sql/execution/row/RowFormatScanRDD.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index 462c795c47..a8843db3b2 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -290,16 +290,21 @@ class RowFormatScanRDD(@transient val session: SnappySession, val startTX = tx eq null // acquire bucket maintenance read lock if required before snapshot gets acquired // and register to be released with transaction commit - val br = BucketRegion.lockPrimaryForMaintenance(false, updateOwner, - container.getRegion.asInstanceOf[PartitionedRegion], bucketIds) + val br = container.getRegion match { + case pr: PartitionedRegion if updateOwner ne null => + BucketRegion.lockPrimaryForMaintenance(false, updateOwner, pr, bucketIds) + case _ => null + } var success = false try { if (startTX) { tx = txManagerImpl.beginTX(TXManagerImpl.getOrCreateTXContext, IsolationLevel.SNAPSHOT, null, null) } - // register the locked region in TX for lock release in case of rollback - success = tx.getProxy.registerLockedBucketRegion(br) + if (br ne null) { + // register the locked region in TX for lock release in case of rollback + success = tx.getProxy.registerLockedBucketRegion(br) + } } finally { if ((br ne null) && !success) { br.unlockAfterMaintenance(false, updateOwner) From 6ec25d09ec89ec433e5a5226f752b01ea9cc7de6 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 3 Aug 2018 12:50:23 +0530 Subject: [PATCH 13/19] fixing putInto which clears context prematurely added an OperationContext in SnappySession that can be used to persist context across multiple plan executions (e.g. caching for putInto, then actual execution) --- .../org/apache/spark/sql/SnappySession.scala | 69 ++++++++++++++++--- .../columnar/JDBCAppendableRelation.scala | 6 +- .../columnar/impl/ColumnFormatRelation.scala | 15 ++-- .../sql/execution/row/RowFormatScanRDD.scala | 9 ++- .../sql/internal/ColumnTableBulkOps.scala | 25 +++++-- 5 files changed, 93 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index dc75721836..cfc47c4146 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -248,6 +248,9 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { @transient private[sql] val contextObjects = new ConcurrentHashMap[Any, Any](16, 0.7f, 1) + @transient + private[sql] var operationContext: Option[OperationContext] = None + @transient private[sql] var currentKey: CachedKey = _ @@ -315,19 +318,47 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { } } - private[sql] def setMutablePlan(mutable: Boolean): Unit = { - if (mutable) { - // use a unique lock owner - val lockOwner = s"MUTABLE_OP_OWNER_$id.${System.nanoTime()}" - addContextObject[String](SnappySession.MUTABLE_PLAN_OWNER, lockOwner) - } else removeContextObject(SnappySession.MUTABLE_PLAN_OWNER) + private[sql] def setMutablePlanOwner(qualifiedTableName: String, + persist: Boolean): Unit = { + if (qualifiedTableName ne null) { + if (persist || qualifiedTableName != getMutablePlanTable) { + val opContext = operationContext match { + case None => + val context = new OperationContext(persist, + ObjectObjectHashMap.withExpectedSize[Any, Any](4)) + operationContext = Some(context) + context.objects + case Some(context) => context.persist = persist; context.objects + } + // use a unique lock owner + val lockOwner = s"MUTABLE_OP_OWNER_$id.${System.nanoTime()}" + opContext.put(SnappySession.MUTABLE_PLAN_TABLE, qualifiedTableName) + opContext.put(SnappySession.MUTABLE_PLAN_OWNER, lockOwner) + } + } else operationContext match { + case None => + case Some(context) => + context.objects.remove(SnappySession.MUTABLE_PLAN_TABLE) + context.objects.remove(SnappySession.MUTABLE_PLAN_OWNER) + } + } + + private[sql] def isMutablePlan: Boolean = operationContext match { + case None => false + case Some(context) => context.objects.containsKey(SnappySession.MUTABLE_PLAN_OWNER) } - private[sql] def isMutablePlan: Boolean = - contextObjects.containsKey(SnappySession.MUTABLE_PLAN_OWNER) + private[sql] def getMutablePlanTable: String = operationContext match { + case None => null + case Some(context) => + context.objects.get(SnappySession.MUTABLE_PLAN_TABLE).asInstanceOf[String] + } - private[sql] def getMutablePlanOwner: String = - contextObjects.get(SnappySession.MUTABLE_PLAN_OWNER).asInstanceOf[String] + private[sql] def getMutablePlanOwner: String = operationContext match { + case None => null + case Some(context) => + context.objects.get(SnappySession.MUTABLE_PLAN_OWNER).asInstanceOf[String] + } def preferPrimaries: Boolean = Property.PreferPrimariesInQuery.get(sessionState.conf) || isMutablePlan @@ -417,6 +448,10 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { sharedState.cacheManager.uncacheQuery(this, cachedPlan, blocking = true) } contextObjects.clear() + operationContext match { + case Some(context) if !context.persist => operationContext = None + case _ => + } planCaching = Property.PlanCaching.get(sessionState.conf) } @@ -1888,7 +1923,9 @@ object SnappySession extends Logging { private[sql] val ExecutionKey = "EXECUTION" private[sql] val CACHED_PUTINTO_UPDATE_PLAN = "cached_putinto_logical_plan" - // internal property to indicate update/delete/putInto execution and lock owner for the same + /** internal property to indicate update/delete/putInto execution and table being mutated */ + private[sql] val MUTABLE_PLAN_TABLE = "snappydata.internal.mutablePlanTable" + /** internal property to indicate update/delete/putInto execution and lock owner for the same */ private[sql] val MUTABLE_PLAN_OWNER = "snappydata.internal.mutablePlanOwner" private[sql] var tokenize: Boolean = _ @@ -2373,3 +2410,13 @@ object CachedKey { new CachedKey(session, currschema, normalizedPlan, sqlText, session.queryHints.hashCode()) } } + +/** + * Encapsulates a context for an operation which can include possibly multiple + * query executions. + * + * @param persist if true then the context is persisted in the session unless explicitly + * cleared else is cleared at end of plan execution like session context + * @param objects key value map of context objects + */ +class OperationContext(var persist: Boolean, val objects: ObjectObjectHashMap[Any, Any]) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala index 9dbf7e2574..4fab289759 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/JDBCAppendableRelation.scala @@ -199,13 +199,13 @@ abstract case class JDBCAppendableRelation( conn.commit() conn.close() } - createExternalTableForColumnBatches(table, externalStore) + createTableForColumnBatches(table, externalStore) } - protected def createExternalTableForColumnBatches(tableName: String, + protected def createTableForColumnBatches(tableName: String, externalStore: ExternalStore): Unit = { require(tableName != null && tableName.length > 0, - "createExternalTableForColumnBatches: expected non-empty table name") + "createTableForColumnBatches: expected non-empty table name") val (primarykey, partitionStrategy) = dialect match { // The driver if not a loner should be an accesor only diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala index 2746dde175..8523b2b52f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnFormatRelation.scala @@ -262,8 +262,10 @@ abstract class BaseColumnFormatRelation( partitioningColumns.map(Utils.toUpperCase) ++ ColumnDelta.mutableKeyNames } - protected def markMutablePlan(): Unit = - sqlContext.sparkSession.asInstanceOf[SnappySession].setMutablePlan(true) + protected def markMutablePlan(): Unit = { + sqlContext.sparkSession.asInstanceOf[SnappySession].setMutablePlanOwner( + resolvedName, persist = false) + } /** * Get a spark plan to update rows in the relation. The result of SparkPlan @@ -392,7 +394,7 @@ abstract class BaseColumnFormatRelation( conn.commit() conn.close() } - createActualTables(table, externalStore) + createActualTables(resolvedName, externalStore) } /** @@ -402,10 +404,10 @@ abstract class BaseColumnFormatRelation( * each for a column. The data column for the base entry will contain the stats. * id for the base entry would be the uuid while for column entries it would be uuid_colName. */ - override def createExternalTableForColumnBatches(tableName: String, + override def createTableForColumnBatches(tableName: String, externalStore: ExternalStore): Unit = { require(tableName != null && tableName.length > 0, - "createExternalTableForColumnBatches: expected non-empty table name") + "createTableForColumnBatches: expected non-empty table name") val (primaryKey, partitionStrategy, concurrency) = dialect match { // The driver if not a loner should be an accessor only @@ -458,8 +460,7 @@ abstract class BaseColumnFormatRelation( case d: JdbcExtendedDialect => d.initializeTable(tableName, sqlContext.conf.caseSensitiveAnalysis, conn) } - createExternalTableForColumnBatches(externalColumnTableName, - externalStore) + createTableForColumnBatches(externalColumnTableName, externalStore) } } catch { case sqle: java.sql.SQLException => diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index a8843db3b2..58eff65ca8 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.row import java.lang.reflect.Field -import java.sql.{Connection, ResultSet, Statement} +import java.sql.{Connection, ResultSet, Statement, Types} import java.util.GregorianCalendar import scala.collection.JavaConverters._ @@ -67,7 +67,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, protected var delayRollover: Boolean, protected var projection: Array[Int]) extends RDDKryo[Any](session.sparkContext, Nil) with KryoSerializable { - protected var updateOwner: String = if (session ne null) session.getMutablePlanOwner else null + protected var updateOwner: String = _ protected var filterWhereArgs: ArrayBuffer[Any] = _ /** * `filters`, but as a WHERE clause suitable for injection into a SQL query. @@ -75,6 +75,9 @@ class RowFormatScanRDD(@transient val session: SnappySession, protected var filterWhereClause: String = _ protected def evaluateWhereClause(): Unit = { + if ((session ne null) && tableName == session.getMutablePlanTable) { + updateOwner = session.getMutablePlanOwner + } val numFilters = filters.length filterWhereClause = if (numFilters > 0) { val sb = new StringBuilder().append(" WHERE ") @@ -209,7 +212,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, } ps.setString(2, bucketString) ps.setInt(3, -1) - ps.setString(4, updateOwner) + if (updateOwner ne null) ps.setString(4, updateOwner) else ps.setNull(4, Types.VARCHAR) ps.executeUpdate() } finally { ps.close() diff --git a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala index 8138ee6133..3793b06e6b 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala @@ -36,16 +36,16 @@ import org.apache.spark.sql.{AnalysisException, Dataset, SnappySession, SparkSes */ object ColumnTableBulkOps { - - def transformPutPlan(sparkSession: SparkSession, originalPlan: PutIntoTable): LogicalPlan = { validateOp(originalPlan) val table = originalPlan.table val subQuery = originalPlan.child var transFormedPlan: LogicalPlan = originalPlan + val session = sparkSession.asInstanceOf[SnappySession] + var success = false table.collectFirst { - case LogicalRelation(mutable: BulkPutRelation, _, _) => + case LogicalRelation(mutable: BulkPutRelation, _, _) => try { val putKeys = mutable.getPutKeys if (putKeys.isEmpty) { throw new AnalysisException( @@ -53,7 +53,7 @@ object ColumnTableBulkOps { } val condition = prepareCondition(sparkSession, table, subQuery, putKeys.get) - val keyColumns = getKeyColumns(table) + val (tableName, keyColumns) = getKeyColumns(table) var updateSubQuery: LogicalPlan = Join(table, subQuery, Inner, condition) val updateColumns = table.output.filterNot(a => keyColumns.contains(a.name)) val updateExpressions = subQuery.output.filterNot(a => keyColumns.contains(a.name)) @@ -67,6 +67,9 @@ object ColumnTableBulkOps { Property.PutIntoInnerJoinCacheSize.get(sparkSession.sqlContext.conf), Property.PutIntoInnerJoinCacheSize.name, -1, Long.MaxValue) + // set a common lock owner for entire operation + session.setMutablePlanOwner(tableName, persist = true) + val updatePlan = Update(table, updateSubQuery, Seq.empty, updateColumns, updateExpressions) val updateDS = new Dataset(sparkSession, updatePlan, RowEncoder(updatePlan.schema)) @@ -94,6 +97,14 @@ object ColumnTableBulkOps { OverwriteOptions(enabled = false), ifNotExists = false) transFormedPlan = PutIntoColumnTable(table, insertPlan, analyzedUpdate) + + // mark operation context as non-persistent at this point so it gets cleared + // after actual execution of transFormedPlan + session.operationContext.get.persist = false + success = true + } finally { + if (!success) session.setMutablePlanOwner(qualifiedTableName = null, persist = false) + } case _ => // Do nothing, original putInto plan is enough } transFormedPlan @@ -143,12 +154,12 @@ object ColumnTableBulkOps { newCondition } - def getKeyColumns(table: LogicalPlan): Seq[String] = { + def getKeyColumns(table: LogicalPlan): (String, Seq[String]) = { table.collectFirst { - case LogicalRelation(mutable: MutableRelation, _, _) => mutable.getKeyColumns + case LogicalRelation(mutable: MutableRelation, _, _) => + mutable.table -> mutable.getKeyColumns }.getOrElse(throw new AnalysisException( s"Update/Delete requires a MutableRelation but got $table")) - } def transformDeletePlan(sparkSession: SparkSession, From 4905afc130983652b88ee313e40cf3222ec9c4fd Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 3 Aug 2018 15:49:07 +0530 Subject: [PATCH 14/19] fix COMMIT call --- .../execution/columnar/impl/JDBCSourceAsColumnarStore.scala | 4 ++-- spark | 2 +- store | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index cc2e5712ca..bce8f63ef6 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -930,8 +930,8 @@ class SmartConnectorRowRDD(_session: SnappySession, ps.setString(1, if (txId ne null) txId else "") ps.setString(2, tableName) ps.setBoolean(3, delayRollover) - ps.setString(3, "") - ps.setInt(4, -1) + ps.setString(4, "") + ps.setInt(5, -1) ps.executeUpdate() logDebug(s"The txid being committed is $txId") ps.close() diff --git a/spark b/spark index e1c758ae2e..93d164d416 160000 --- a/spark +++ b/spark @@ -1 +1 @@ -Subproject commit e1c758ae2eebdf4c9511d2b967420af0fe330154 +Subproject commit 93d164d41634c688bc5125bef5dc96118ea7db35 diff --git a/store b/store index cfce645e62..b233177867 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit cfce645e622fb777cbdae27bbfb1417e55220e42 +Subproject commit b2331778672c1ccb84ae7959d911d4148001fdc2 From 778bb4b1be073c9f6aacdb0f485a529a49edc4ac Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 3 Aug 2018 17:16:42 +0530 Subject: [PATCH 15/19] update store link --- .../main/scala/org/apache/spark/sql/SnappySession.scala | 9 ++++++--- store | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index cfc47c4146..ae2d0fecfa 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -102,9 +102,12 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { SnappyContext.getClusterMode(sparkContext) match { case _: SnappyEmbeddedMode => val deployCmds = ToolsCallbackInit.toolsCallback.getAllGlobalCmnds() - logInfo(s"deployCmnds size = ${deployCmds.length}") - logDebug(s"deployCmds = ${deployCmds.mkString(", ")}") - deployCmds.foreach(d => { + val nonEmpty = deployCmds.length > 0 + if (nonEmpty) { + logInfo(s"deployCmnds size = ${deployCmds.length}") + logDebug(s"deployCmds = ${deployCmds.mkString(", ")}") + } + if (nonEmpty) deployCmds.foreach(d => { val cmdFields = d.split('|') if (cmdFields.length > 1) { val coordinate = cmdFields(0) diff --git a/store b/store index b233177867..4564b4c3b5 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit b2331778672c1ccb84ae7959d911d4148001fdc2 +Subproject commit 4564b4c3b5c58413f38ae33ce1290cea1c6fe32c From 647d243429687cfd15463122dd095b5815a507ff Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Mon, 6 Aug 2018 11:54:59 +0530 Subject: [PATCH 16/19] updates and fixes --- .../cluster/SplitSnappyClusterDUnitTest.scala | 14 ++-- .../cluster/SplitClusterDUnitTestBase.scala | 14 ++-- .../main/scala/io/snappydata/functions.scala | 67 ++++++++++++++++- .../apache/spark/sql/SnappyDDLParser.scala | 15 +++- .../org/apache/spark/sql/SnappyParser.scala | 4 +- .../org/apache/spark/sql/SnappySession.scala | 5 +- .../spark/sql/execution/ExistingPlans.scala | 29 +++++++- .../sql/execution/columnar/ColumnExec.scala | 7 +- .../execution/columnar/ColumnInsertExec.scala | 8 +- .../impl/JDBCSourceAsColumnarStore.scala | 74 +++++++------------ .../sql/execution/row/RowFormatScanRDD.scala | 29 +++----- .../sql/internal/ColumnTableBulkOps.scala | 48 +++++++++++- .../snappydata/ColumnUpdateDeleteTests.scala | 61 +++++++-------- store | 2 +- 14 files changed, 242 insertions(+), 135 deletions(-) diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala index 3ac629a951..b558c33a73 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala @@ -260,13 +260,13 @@ class SplitSnappyClusterDUnitTest(s: String) // added in SNAP-2012 StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = true try { - ColumnUpdateDeleteTests.testBasicUpdate(session) - ColumnUpdateDeleteTests.testDeltaStats(session) - ColumnUpdateDeleteTests.testBasicDelete(session) - ColumnUpdateDeleteTests.testSNAP1925(session) - ColumnUpdateDeleteTests.testSNAP1926(session) - ColumnUpdateDeleteTests.testConcurrentOps(session) - ColumnUpdateDeleteTests.testSNAP2124(session, checkPruning = true) + ColumnUpdateDeleteTests.testBasicUpdate(session, redundancy = 1) + ColumnUpdateDeleteTests.testDeltaStats(session, redundancy = 1) + ColumnUpdateDeleteTests.testBasicDelete(session, redundancy = 1) + ColumnUpdateDeleteTests.testSNAP1925(session, redundancy = 1) + ColumnUpdateDeleteTests.testSNAP1926(session, redundancy = 1) + ColumnUpdateDeleteTests.testConcurrentOps(session, redundancy = 1) + ColumnUpdateDeleteTests.testSNAP2124(session, checkPruning = true, redundancy = 1) } finally { StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = false } diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala index 0239fe150b..09d850764d 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala @@ -205,13 +205,13 @@ trait SplitClusterDUnitTestBase extends Logging { // using random bucket assignment for cases like SNAP-2175 StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = true try { - ColumnUpdateDeleteTests.testBasicUpdate(session) - ColumnUpdateDeleteTests.testDeltaStats(session) - ColumnUpdateDeleteTests.testBasicDelete(session) - ColumnUpdateDeleteTests.testSNAP1925(session) - ColumnUpdateDeleteTests.testSNAP1926(session) - ColumnUpdateDeleteTests.testConcurrentOps(session) - ColumnUpdateDeleteTests.testSNAP2124(session, checkPruning = true) + ColumnUpdateDeleteTests.testBasicUpdate(session, redundancy = 1) + ColumnUpdateDeleteTests.testDeltaStats(session, redundancy = 1) + ColumnUpdateDeleteTests.testBasicDelete(session, redundancy = 1) + ColumnUpdateDeleteTests.testSNAP1925(session, redundancy = 1) + ColumnUpdateDeleteTests.testSNAP1926(session, redundancy = 1) + ColumnUpdateDeleteTests.testConcurrentOps(session, redundancy = 1) + ColumnUpdateDeleteTests.testSNAP2124(session, checkPruning = true, redundancy = 1) } finally { StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = false } diff --git a/core/src/main/scala/io/snappydata/functions.scala b/core/src/main/scala/io/snappydata/functions.scala index fc463bfa8f..e2fb17435a 100644 --- a/core/src/main/scala/io/snappydata/functions.scala +++ b/core/src/main/scala/io/snappydata/functions.scala @@ -31,11 +31,70 @@ import org.apache.spark.unsafe.types.UTF8String */ object SnappyDataFunctions { - val usageStr = "_FUNC_() - Returns the unique distributed member" + - " ID of the server containing the row." - def registerSnappyFunctions(functionRegistry: FunctionRegistry): Unit = { - val info = new ExpressionInfo(DSID.getClass.getCanonicalName, null, "DSID", usageStr, "") + var usageStr = "" + var extendedStr = "" + var info: ExpressionInfo = null + + // below are in-built operators additionally handled in snappydata over spark + // which are listed so they can appear in describe function + + // --- BEGIN OPERATORS --- + + usageStr = "expr1 _FUNC_ expr2 - Bitwise left shift `expr1` by `expr2`." + extendedStr = """ + Examples: + > SELECT 15 _FUNC_ 2; + 60 + """ + info = new ExpressionInfo("", null, "<<", usageStr, extendedStr) + + usageStr = "expr1 _FUNC_ expr2 - Bitwise arithmetic right shift `expr1` by `expr2`." + extendedStr = """ + Examples: + > SELECT 15 _FUNC_ 2; + 3 + > SELECT -15 _FUNC_ 2; + -4 + """ + info = new ExpressionInfo("", null, ">>", usageStr, extendedStr) + + usageStr = "expr1 _FUNC_ expr2 - Bitwise logical right shift `expr1` by `expr2`." + extendedStr = """ + Examples: + > SELECT 15 _FUNC_ 2; + 3 + > SELECT -15 _FUNC_ 2; + 1073741820 + """ + info = new ExpressionInfo("", null, ">>>", usageStr, extendedStr) + + usageStr = "str1 || str2 - Returns the concatenation of str1 and str2." + extendedStr = """ + Examples: + > SELECT 'Spark' _FUNC_ 'SQL'; + SparkSQL + """ + info = new ExpressionInfo("", null, "||", usageStr, extendedStr) + + // --- END OPERATORS --- + + usageStr = "_FUNC_() - Returns the unique distributed member " + + "ID of the server containing the current row being fetched." + extendedStr = """ + Examples: + > SELECT _FUNC_, ID FROM RANGE(1, 10); + 127.0.0.1(25167):16171|1 + 127.0.0.1(25167):16171|2 + 127.0.0.1(25167):16171|3 + 127.0.0.1(25167):16171|4 + 127.0.0.1(25078):13152|5 + 127.0.0.1(25078):13152|6 + 127.0.0.1(25078):13152|7 + 127.0.0.1(25078):13152|8 + 127.0.0.1(25167):16171|9 + """ + info = new ExpressionInfo(DSID.getClass.getCanonicalName, null, "DSID", usageStr, extendedStr) functionRegistry.registerFunction("DSID", info, _ => DSID()) } } diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala index ddae382fed..607d040add 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala @@ -549,6 +549,19 @@ abstract class SnappyDDLParser(session: SparkSession) .asInstanceOf[Option[Boolean]].isDefined, isFormatted = false)) } + protected def describeFunction: Rule1[LogicalPlan] = rule { + DESCRIBE ~ (EXTENDED ~ push(true)).? ~ (functionIdentifier | stringLiteral | + capture("==" | "=" | "!=" | "<>" | ">=" | ">>" | ">>>" | ">" | "<=>" | "<=" | "<<" | "<" | + "+" | "-" | "*" | "/" | "%" | "~" | "&" | "||" | "|" | "~" | + OR | AND | IN | NOT)) ~> { (extended: Any, ident: Any) => + val functionIdent = ident match { + case f: FunctionIdentifier => f + case s: String => FunctionIdentifier(s, database = None) + } + DescribeFunctionCommand(functionIdent, extended.asInstanceOf[Option[Boolean]].isDefined) + } + } + protected def refreshTable: Rule1[LogicalPlan] = rule { REFRESH ~ TABLE ~ tableIdentifier ~> RefreshTable } @@ -810,7 +823,7 @@ case class DeployCommand( Misc.checkIfCacheClosing(ex) if (restart) { logWarning(s"Following mvn coordinate" + - s" could not be resolved during restart: ${coordinates}", ex) + s" could not be resolved during restart: $coordinates", ex) if (lang.Boolean.parseBoolean(System.getProperty("FAIL_ON_JAR_UNAVAILABILITY", "true"))) { throw ex } diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala b/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala index 3f8fe7a142..8b30adf6c6 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyParser.scala @@ -373,7 +373,9 @@ class SnappyParser(session: SnappySession) protected final def comparisonExpression: Rule1[Expression] = rule { termExpression ~ ( - '=' ~ ws ~ termExpression ~> EqualTo | + '=' ~ ( + '=' ~ ws ~ termExpression ~> EqualTo | + ws ~ termExpression ~> EqualTo) | '>' ~ ( '=' ~ ws ~ termExpression ~> GreaterThanOrEqual | '>' ~ ( diff --git a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala index ae2d0fecfa..c60620e978 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -334,7 +334,7 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { case Some(context) => context.persist = persist; context.objects } // use a unique lock owner - val lockOwner = s"MUTABLE_OP_OWNER_$id.${System.nanoTime()}" + val lockOwner = s"READ_${SnappySession.MUTABLE_OWNER_PREFIX}_$id.${System.nanoTime()}" opContext.put(SnappySession.MUTABLE_PLAN_TABLE, qualifiedTableName) opContext.put(SnappySession.MUTABLE_PLAN_OWNER, lockOwner) } @@ -1931,6 +1931,9 @@ object SnappySession extends Logging { /** internal property to indicate update/delete/putInto execution and lock owner for the same */ private[sql] val MUTABLE_PLAN_OWNER = "snappydata.internal.mutablePlanOwner" + /** a unique UUID of the node for mutability lock ownership */ + private[sql] lazy val MUTABLE_OWNER_PREFIX = java.util.UUID.randomUUID().toString + private[sql] var tokenize: Boolean = _ lazy val isEnterpriseEdition: Boolean = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala index 1f9bcef67c..b49d98722a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala @@ -30,10 +30,11 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, IndexColumnFormatRelation} -import org.apache.spark.sql.execution.columnar.{ColumnTableScan, ConnectionType} +import org.apache.spark.sql.execution.columnar.{ColumnDeleteExec, ColumnPutIntoExec, ColumnTableScan, ColumnUpdateExec, ConnectionType} import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetricInfo, SQLMetrics} import org.apache.spark.sql.execution.row.{RowFormatRelation, RowTableScan} +import org.apache.spark.sql.internal.ColumnTableBulkOps import org.apache.spark.sql.sources.{BaseRelation, PrunedUnsafeFilteredScan, SamplingRelation} import org.apache.spark.sql.types._ import org.apache.spark.sql.{AnalysisException, CachedDataFrame, SnappySession} @@ -235,7 +236,33 @@ case class ExecutePlan(child: SparkPlan, preAction: () => Unit = () => ()) Array.concat(rows: _*) } + private val (mutableTable, lockOwner) = { + val session = sqlContext.sparkSession.asInstanceOf[SnappySession] + session.getMutablePlanTable match { + case null => (null, null) + case table => (table, session.getMutablePlanOwner) + } + } + protected[sql] lazy val sideEffectResult: Array[InternalRow] = { + try { + getSideEffectResult + } finally { + // release locks at the end of update/delete/putInto + if ((mutableTable ne null) && (lockOwner ne null)) { + ColumnTableBulkOps.releaseBucketMaintenanceLocks(mutableTable, lockOwner, () => + SnappySession.getExecutedPlan(child) match { + case (u: ColumnUpdateExec, _) => u.connProps + case (d: ColumnDeleteExec, _) => d.connProps + case (p: ColumnPutIntoExec, _) => p.updatePlan.asInstanceOf[ColumnUpdateExec].connProps + case _ => throw new IllegalStateException( + s"Unexpected plan for ${child.getClass.getName}: $child") + }, sparkContext) + } + } + } + + private def getSideEffectResult: Array[InternalRow] = { val session = sqlContext.sparkSession.asInstanceOf[SnappySession] val sc = session.sparkContext val key = session.currentKey diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala index 3b3b7ad013..f8e0f3dc75 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnExec.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.columnar import java.sql.Connection -import org.apache.spark.sql.SnappySession import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.{JDBCSourceAsColumnarStore, SnapshotConnectionListener} @@ -40,10 +39,7 @@ trait ColumnExec extends RowExec { override protected def connectionCodes(ctx: CodegenContext): (String, String, String) = { val connectionClass = classOf[Connection].getName - val session = sqlContext.sparkSession.asInstanceOf[SnappySession] val externalStoreTerm = ctx.addReferenceObj("externalStore", externalStore) - val updateOwner = ctx.addReferenceObj("updateOwner", - session.getMutablePlanOwner, classOf[String].getName) val listenerClass = classOf[SnapshotConnectionListener].getName val storeClass = classOf[JDBCSourceAsColumnarStore].getName taskListener = ctx.freshName("taskListener") @@ -55,8 +51,7 @@ trait ColumnExec extends RowExec { val initCode = s""" - |$taskListener = new $listenerClass(($storeClass)$externalStoreTerm, - | $delayRollover, $updateOwner); + |$taskListener = new $listenerClass(($storeClass)$externalStoreTerm, $delayRollover); |$connTerm = $taskListener.getConn(); |if ($getContext() != null) { | $getContext().addTaskCompletionListener($taskListener); diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala index 9957f8e2ae..170adcbfa0 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/ColumnInsertExec.scala @@ -588,14 +588,14 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String], ctx.addNewFunction(commitSnapshotTx, s""" |private final void $commitSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.commitTx($txId, false, null, -1, $conn); + | $externalStoreTerm.commitTx($txId, false, $conn); |} """.stripMargin) rollbackSnapshotTx = ctx.freshName("rollbackSnapshotTx") ctx.addNewFunction(rollbackSnapshotTx, s""" |private final void $rollbackSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.rollbackTx($txId, null, -1, $conn); + | $externalStoreTerm.rollbackTx($txId, $conn); |} """.stripMargin) closeConnection = ctx.freshName("closeConnection") @@ -725,14 +725,14 @@ case class ColumnInsertExec(child: SparkPlan, partitionColumns: Seq[String], ctx.addNewFunction(commitSnapshotTx, s""" |private final void $commitSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.commitTx($txId, false, null, -1, $conn); + | $externalStoreTerm.commitTx($txId, false, $conn); |} """.stripMargin) rollbackSnapshotTx = ctx.freshName("rollbackSnapshotTx") ctx.addNewFunction(rollbackSnapshotTx, s""" |private final void $rollbackSnapshotTx(String $txId, scala.Option $conn) { - | $externalStoreTerm.rollbackTx($txId, null, -1, $conn); + | $externalStoreTerm.rollbackTx($txId, $conn); |} """.stripMargin) closeConnection = ctx.freshName("closeConnection") diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index bce8f63ef6..da8dd6274e 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -163,32 +163,22 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie }(Some(conn)) } - def commitTx(txId: String, delayRollover: Boolean, updateOwner: String, - partitionId: Int, conn: Option[Connection]): Unit = { + def commitTx(txId: String, delayRollover: Boolean, conn: Option[Connection]): Unit = { tryExecute(tableName, closeOnSuccessOrFailure = false, onExecutor = true)(conn => { var success = false try { connectionType match { - case ConnectionType.Embedded => try { + case ConnectionType.Embedded => // if rollover was marked as delayed, then do the rollover before commit if (delayRollover) { GfxdSystemProcedures.flushLocalBuckets(tableName, false) } Misc.getGemFireCache.getCacheTransactionManager.commit() - } finally { - if (updateOwner ne null) { - GfxdSystemProcedures.unlockBucketAfterMaintenance( - tableName, updateOwner, partitionId) - } - } case _ => logDebug(s"Going to commit $txId the transaction on server conn is $conn") - val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?,?,?,?)") + val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, tableName) - ps.setBoolean(3, delayRollover) - ps.setString(4, if (updateOwner ne null) updateOwner else "") - ps.setInt(5, partitionId) + ps.setString(2, if (delayRollover) tableName else "") try { ps.executeUpdate() logDebug(s"The txid being committed is $txId") @@ -213,30 +203,19 @@ class JDBCSourceAsColumnarStore(private var _connProperties: ConnectionPropertie } - def rollbackTx(txId: String, updateOwner: String, partitionId: Int, - conn: Option[Connection]): Unit = { + def rollbackTx(txId: String, conn: Option[Connection]): Unit = { // noinspection RedundantDefaultArgument tryExecute(tableName, closeOnSuccessOrFailure = true, onExecutor = true) { (conn: Connection) => { connectionType match { case ConnectionType.Embedded => - try { - Misc.getGemFireCache.getCacheTransactionManager.rollback() - } finally { - if (updateOwner ne null) { - GfxdSystemProcedures.unlockBucketAfterMaintenance( - tableName, updateOwner, partitionId) - } - } + Misc.getGemFireCache.getCacheTransactionManager.rollback() case _ => logDebug(s"Going to rollback transaction $txId on server using $conn") var ps: PreparedStatement = null handleRollback(() => { - ps = conn.prepareStatement(s"call sys.ROLLBACK_SNAPSHOT_TXID(?,?,?,?)") + ps = conn.prepareStatement(s"call sys.ROLLBACK_SNAPSHOT_TXID(?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, tableName) - ps.setString(3, if (updateOwner ne null) updateOwner else "") - ps.setInt(4, partitionId) ps.executeUpdate() logDebug(s"The transaction ID being rolled back is $txId") }, () => { @@ -819,12 +798,9 @@ final class SmartConnectorColumnRDD( logDebug(s"The txid going to be committed is $txId " + tableName) // if ((txId ne null) && !txId.equals("null")) { - val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?,?,?,?)") + val ps = conn.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, tableName) - ps.setBoolean(3, delayRollover) - ps.setString(4, "") - ps.setInt(5, -1) + ps.setString(2, if (delayRollover) tableName else "") ps.executeUpdate() logDebug(s"The txid being committed is $txId") ps.close() @@ -926,12 +902,9 @@ class SmartConnectorRowRDD(_session: SnappySession, val txId = SmartConnectorRDDHelper.snapshotTxIdForRead.get logDebug(s"The txid going to be committed is $txId " + tableName) // if ((txId ne null) && !txId.equals("null")) { - val ps = conn.get.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?,?,?,?)") + val ps = conn.get.prepareStatement(s"call sys.COMMIT_SNAPSHOT_TXID(?,?)") ps.setString(1, if (txId ne null) txId else "") - ps.setString(2, tableName) - ps.setBoolean(3, delayRollover) - ps.setString(4, "") - ps.setInt(5, -1) + ps.setString(2, if (delayRollover) tableName else "") ps.executeUpdate() logDebug(s"The txid being committed is $txId") ps.close() @@ -948,10 +921,10 @@ class SmartConnectorRowRDD(_session: SnappySession, if (context ne null) { val partitionId = context.partitionId() context.addTaskCompletionListener { _ => - logDebug(s"closed connection for task from listener $partitionId") + logDebug(s"closing connection for task from listener $partitionId") try { conn.close() - logDebug("closed connection for task " + context.partitionId()) + logDebug(s"closed connection for task $partitionId partition = $thePart") } catch { case NonFatal(e) => logWarning("Exception closing connection", e) } @@ -1035,19 +1008,22 @@ class SmartConnectorRowRDD(_session: SnappySession, } class SnapshotConnectionListener(store: JDBCSourceAsColumnarStore, - delayRollover: Boolean, updateOwner: String) extends TaskCompletionListener { - private val connAndTxId: Array[_ <: Object] = store.beginTx(delayRollover) - private var isSuccess = false + delayRollover: Boolean) extends TaskCompletionListener { + val connAndTxId: Array[_ <: Object] = store.beginTx(delayRollover) + var isSuccess = false override def onTaskCompletion(context: TaskContext): Unit = { - val conn = Option(connAndTxId(0).asInstanceOf[Connection]) val txId = connAndTxId(1).asInstanceOf[String] - if (success()) { - store.commitTx(txId, delayRollover, updateOwner, context.partitionId(), conn) - } else { - store.rollbackTx(txId, updateOwner, context.partitionId(), conn) + val conn = connAndTxId(0).asInstanceOf[Connection] + if (connAndTxId(1) != null) { + if (success()) { + store.commitTx(txId, delayRollover, Some(conn)) + } + else { + store.rollbackTx(txId, Some(conn)) + } } - store.closeConnection(conn) + store.closeConnection(Some(conn)) } def success(): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala index 58eff65ca8..c8737f55d6 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/row/RowFormatScanRDD.scala @@ -30,6 +30,7 @@ import com.gemstone.gemfire.cache.IsolationLevel import com.gemstone.gemfire.internal.cache._ import com.gemstone.gemfire.internal.shared.ClientSharedData import com.pivotal.gemfirexd.internal.engine.Misc +import com.pivotal.gemfirexd.internal.engine.access.GfxdTXStateProxy import com.pivotal.gemfirexd.internal.engine.ddl.catalog.GfxdSystemProcedures import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils import com.pivotal.gemfirexd.internal.engine.store.{AbstractCompactExecRow, GemFireContainer, RawStoreResultSet, RegionEntryUtils} @@ -190,11 +191,11 @@ class RowFormatScanRDD(@transient val session: SnappySession, if (context ne null) { val partitionId = context.partitionId() context.addTaskCompletionListener { _ => - logDebug(s"closed connection for task from listener $partitionId") + logDebug(s"closing connection for task from listener $partitionId") try { conn.commit() conn.close() - logDebug("closed connection for task " + context.partitionId()) + logDebug(s"closed connection for task $partitionId partition = $thePart") } catch { case NonFatal(e) => logWarning("Exception closing connection", e) } @@ -292,26 +293,14 @@ class RowFormatScanRDD(@transient val session: SnappySession, var tx = txManagerImpl.getTXState val startTX = tx eq null // acquire bucket maintenance read lock if required before snapshot gets acquired - // and register to be released with transaction commit - val br = container.getRegion match { + container.getRegion match { case pr: PartitionedRegion if updateOwner ne null => - BucketRegion.lockPrimaryForMaintenance(false, updateOwner, pr, bucketIds) - case _ => null + GfxdSystemProcedures.lockPrimaryForMaintenance(false, updateOwner, pr, bucketIds) + case _ => } - var success = false - try { - if (startTX) { - tx = txManagerImpl.beginTX(TXManagerImpl.getOrCreateTXContext, - IsolationLevel.SNAPSHOT, null, null) - } - if (br ne null) { - // register the locked region in TX for lock release in case of rollback - success = tx.getProxy.registerLockedBucketRegion(br) - } - } finally { - if ((br ne null) && !success) { - br.unlockAfterMaintenance(false, updateOwner) - } + if (startTX) { + tx = txManagerImpl.beginTX(TXManagerImpl.getOrCreateTXContext, + IsolationLevel.SNAPSHOT, null, null) } // use iterator over CompactExecRows directly when no projection; // higher layer PartitionedPhysicalRDD will take care of conversion diff --git a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala index 3793b06e6b..3a83f1d5ab 100644 --- a/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala +++ b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala @@ -16,18 +16,21 @@ */ package org.apache.spark.sql.internal +import com.pivotal.gemfirexd.internal.engine.ddl.catalog.GfxdSystemProcedures import io.snappydata.Property +import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, EqualTo, Expression} import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, OverwriteOptions, Project} import org.apache.spark.sql.catalyst.plans.{Inner, LeftAnti} import org.apache.spark.sql.collection.Utils -import org.apache.spark.sql.execution.columnar.ExternalStoreUtils +import org.apache.spark.sql.execution.ConnectionPool +import org.apache.spark.sql.execution.columnar.{ExternalStoreUtils, JDBCAppendableRelation} import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, LongType} -import org.apache.spark.sql.{AnalysisException, Dataset, SnappySession, SparkSession} +import org.apache.spark.sql.{AnalysisException, Dataset, SnappyContext, SnappySession, SparkSession, ThinClientConnectorMode} /** * Helper object for PutInto operations for column tables. @@ -43,6 +46,7 @@ object ColumnTableBulkOps { var transFormedPlan: LogicalPlan = originalPlan val session = sparkSession.asInstanceOf[SnappySession] var success = false + var tableName: String = null table.collectFirst { case LogicalRelation(mutable: BulkPutRelation, _, _) => try { @@ -53,7 +57,8 @@ object ColumnTableBulkOps { } val condition = prepareCondition(sparkSession, table, subQuery, putKeys.get) - val (tableName, keyColumns) = getKeyColumns(table) + val (tName, keyColumns) = getKeyColumns(table) + tableName = tName var updateSubQuery: LogicalPlan = Join(table, subQuery, Inner, condition) val updateColumns = table.output.filterNot(a => keyColumns.contains(a.name)) val updateExpressions = subQuery.output.filterNot(a => keyColumns.contains(a.name)) @@ -103,13 +108,48 @@ object ColumnTableBulkOps { session.operationContext.get.persist = false success = true } finally { - if (!success) session.setMutablePlanOwner(qualifiedTableName = null, persist = false) + if (!success) { + val lockOwner = session.getMutablePlanOwner + if ((tableName ne null) && (lockOwner ne null)) { + releaseBucketMaintenanceLocks(tableName, lockOwner, () => { + // lookup catalog and get the properties from column table relation + val catalog = session.sessionCatalog + val relation = catalog.lookupRelation(catalog.newQualifiedTableName(tableName)) + relation.asInstanceOf[JDBCAppendableRelation].externalStore.connProperties + }, session.sparkContext) + } + session.setMutablePlanOwner(qualifiedTableName = null, persist = false) + } } case _ => // Do nothing, original putInto plan is enough } transFormedPlan } + def releaseBucketMaintenanceLocks(tableName: String, lockOwner: String, + getConnProps: () => ConnectionProperties, sparkContext: SparkContext): Unit = { + SnappyContext.getClusterMode(sparkContext) match { + case ThinClientConnectorMode(_, _) => + // get the connection properties + val connProps = getConnProps() + val conn = ConnectionPool.getPoolConnection(tableName, connProps.dialect, + connProps.poolProps, connProps.connProps, connProps.hikariCP) + try { + val stmt = conn.prepareCall("call SYS.RELEASE_BUCKET_MAINTENANCE_LOCKS(?,?,?,?)") + stmt.setString(1, tableName) + stmt.setBoolean(2, false) + stmt.setString(3, lockOwner) + stmt.setNull(4, java.sql.Types.VARCHAR) + stmt.execute() + stmt.close() + } finally { + conn.close() + } + case _ => GfxdSystemProcedures.releaseBucketMaintenanceLocks( + tableName, false, lockOwner, null) + } + } + def validateOp(originalPlan: PutIntoTable) { originalPlan match { case PutIntoTable(LogicalRelation(t: BulkPutRelation, _, _), query) => diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index b9d6567b74..559546134d 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -24,7 +24,6 @@ import scala.concurrent.duration.Duration import scala.concurrent.{Await, ExecutionContext, Future} import com.gemstone.gemfire.internal.cache.{GemFireCacheImpl, PartitionedRegion} -import com.pivotal.gemfirexd.TestUtil import com.pivotal.gemfirexd.internal.engine.Misc import com.pivotal.gemfirexd.internal.engine.store.GemFireContainer import io.snappydata.SnappyFunSuite.checkAnswer @@ -39,7 +38,10 @@ import org.apache.spark.sql.{Row, SnappySession} */ object ColumnUpdateDeleteTests extends Assertions with Logging { - def testBasicUpdate(session: SnappySession): Unit = { + private def ddlExt(redundancy: Int): String = + if (redundancy > 0) s", redundancy '$redundancy'" else "" + + def testBasicUpdate(session: SnappySession, redundancy: Int = 0): Unit = { session.conf.set(Property.ColumnBatchSize.name, "10k") // session.conf.set(Property.ColumnMaxDeltaRows.name, "200") @@ -50,14 +52,15 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.sql("drop table if exists checkTable2") session.sql("drop table if exists checkTable3") + val ext = ddlExt(redundancy) session.sql("create table updateTable (id int, addr string, status boolean) " + - "using column options(buckets '4')") + s"using column options(buckets '4'$ext)") session.sql("create table checkTable1 (id int, addr string, status boolean) " + - "using column options(buckets '4')") + s"using column options(buckets '4'$ext)") session.sql("create table checkTable2 (id int, addr string, status boolean) " + - "using column options(buckets '2')") + s"using column options(buckets '2'$ext)") session.sql("create table checkTable3 (id int, addr string, status boolean) " + - "using column options(buckets '1')") + s"using column options(buckets '1'$ext)") session.range(numElements).selectExpr("id", "concat('addr', cast(id as string))", @@ -200,10 +203,11 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.conf.unset(Property.ColumnBatchSize.name) } - def testDeltaStats(session: SnappySession): Unit = { + def testDeltaStats(session: SnappySession, redundancy: Int = 0): Unit = { session.sql("drop table if exists test1") + val ext = ddlExt(redundancy) session.sql("create table test1 (col1 long, col2 long) using column " + - "options (buckets '1', column_batch_size '50')") + s"options (buckets '1', column_batch_size '50'$ext)") // size of batch ensured so that both rows fall in same batch session.range(2).selectExpr("(id + 1) * 10", "(id + 1) * 100").write.insertInto("test1") @@ -247,7 +251,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.sql("drop table test1") } - def testBasicDelete(session: SnappySession): Unit = { + def testBasicDelete(session: SnappySession, redundancy: Int = 0): Unit = { session.conf.set(Property.ColumnBatchSize.name, "10k") // session.conf.set(Property.ColumnMaxDeltaRows.name, "200") @@ -256,14 +260,15 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.sql("drop table if exists checkTable2") session.sql("drop table if exists checkTable3") + val ext = ddlExt(redundancy) session.sql("create table updateTable (id int, addr string, status boolean) " + - "using column options(buckets '4', partition_by 'addr')") + s"using column options(buckets '4', partition_by 'addr'$ext)") session.sql("create table checkTable1 (id int, addr string, status boolean) " + - "using column options(buckets '2')") + s"using column options(buckets '2'$ext)") session.sql("create table checkTable2 (id int, addr string, status boolean) " + - "using column options(buckets '8')") + s"using column options(buckets '8'$ext)") session.sql("create table checkTable3 (id int, addr string, status boolean) " + - "using column options(buckets '2')") + s"using column options(buckets '2'$ext)") for (_ <- 1 to 3) { testBasicDeleteIter(session) @@ -360,16 +365,17 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { assert(session.sql("select * from updateTable").collect().length === 0) } - def testSNAP1925(session: SnappySession): Unit = { + def testSNAP1925(session: SnappySession, redundancy: Int = 0): Unit = { // reduced size to ensure both column table and row buffer have data session.conf.set(Property.ColumnBatchSize.name, "10k") val numElements = 50000 + val ext = ddlExt(redundancy) session.sql("drop table if exists order_details") session.sql("create table order_details (OrderID int, ProductID int," + "UnitPrice double, Quantity smallint, Discount double, tid int) " + - "using column options(partition_by 'OrderID', buckets '8')") + s"using column options(partition_by 'OrderID', buckets '8'$ext)") session.range(numElements).selectExpr("id", "id + 2", "1.0", "2", "rand()", "id + 1") .write.insertInto("order_details") @@ -413,18 +419,19 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.conf.unset(Property.ColumnBatchSize.name) } - def testSNAP1926(session: SnappySession): Unit = { + def testSNAP1926(session: SnappySession, redundancy: Int = 0): Unit = { // reduced size to ensure both column table and row buffer have data session.conf.set(Property.ColumnBatchSize.name, "10k") val numElements = 50000 session.sql("drop table if exists customers") + val ext = ddlExt(redundancy) session.sql("CREATE TABLE CUSTOMERS (CUSTOMERID VARCHAR(100), COMPANYNAME VARCHAR(100), " + "CONTACTNAME VARCHAR(100), CONTACTTITLE VARCHAR(100), ADDRESS VARCHAR(100), " + "CITY VARCHAR(100), REGION VARCHAR(100), POSTALCODE VARCHAR(100), " + "COUNTRY VARCHAR(100), PHONE VARCHAR(100), FAX VARCHAR(100), TID INTEGER) " + - "using column options(partition_by 'City,Country', buckets '8')") + s"using column options(partition_by 'City,Country', buckets '8'$ext)") session.range(numElements).selectExpr("id", "id + 1", "id + 2", "id + 3", "id + 4", "id + 5", "id + 6", "id + 7", "id + 8", "id + 9", "id + 10", "id % 20") @@ -442,13 +449,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.conf.unset(Property.ColumnBatchSize.name) } - def testConcurrentOps(session: SnappySession): Unit = { - // start network server - val serverHostPort = TestUtil.startNetServer() - // scalastyle:off println - println(s"Started network server on $serverHostPort") - // scalastyle:on println - + def testConcurrentOps(session: SnappySession, redundancy: Int = 0): Unit = { // reduced size to ensure both column table and row buffer have data session.conf.set(Property.ColumnBatchSize.name, "10k") // session.conf.set(Property.ColumnMaxDeltaRows.name, "200") @@ -458,12 +459,13 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.sql("drop table if exists checkTable2") session.sql("drop table if exists checkTable3") + val ext = ddlExt(redundancy) session.sql("create table updateTable (id int, addr string, status boolean) " + - "using column options(buckets '4')") + s"using column options(buckets '4'$ext)") session.sql("create table checkTable1 (id int, addr string, status boolean) " + - "using column options(buckets '2')") + s"using column options(buckets '2'$ext)") session.sql("create table checkTable2 (id int, addr string, status boolean) " + - "using column options(buckets '8')") + s"using column options(buckets '8'$ext)") // avoid rollover in updateTable during concurrent updates val avoidRollover = new SerializableRunnable() { @@ -599,14 +601,15 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { } } - def testSNAP2124(session: SnappySession, checkPruning: Boolean): Unit = { + def testSNAP2124(session: SnappySession, checkPruning: Boolean, redundancy: Int = 0): Unit = { val filePath = getClass.getResource("/sample_records.json").getPath + val ext = ddlExt(redundancy) session.sql("CREATE TABLE domaindata (cntno_l string,cntno_m string," + "day1 string,day2 string,day3 string,day4 string,day5 string," + "day6 string,day7 string,dr string,ds string,email string," + "id BIGINT NOT NULL,idinfo_1 string,idinfo_2 string,idinfo_3 string," + "idinfo_4 string,lang_1 string,lang_2 string,lang_3 string,name string) " + - "USING COLUMN OPTIONS (PARTITION_BY 'id',BUCKETS '40', COLUMN_BATCH_SIZE '10')") + s"USING COLUMN OPTIONS (PARTITION_BY 'id',BUCKETS '40', COLUMN_BATCH_SIZE '10'$ext)") session.read.json(filePath).write.insertInto("domaindata") var ds = session.sql("select ds, dr from domaindata where id = 40L") diff --git a/store b/store index 4564b4c3b5..540063d726 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 4564b4c3b5c58413f38ae33ce1290cea1c6fe32c +Subproject commit 540063d7263038cb3d67bc22297c87889945e0d3 From 94c45d68c4a3ac357c0f643e290ea4396b5c39b8 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 10 Aug 2018 11:42:50 +0530 Subject: [PATCH 17/19] fix build issues after master merge --- .../org/apache/spark/sql/aqp/SnappyContextFunctions.scala | 4 +--- .../org/apache/spark/sql/execution/ExistingPlans.scala | 6 ++++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala b/core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala index d0b507f4c3..605ba446fe 100644 --- a/core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala +++ b/core/src/main/scala/org/apache/spark/sql/aqp/SnappyContextFunctions.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.aqp -import io.snappydata.SnappyDataFunctions.usageStr - import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -46,7 +44,7 @@ class SnappyContextFunctions { "CURRENT_USER", usageStr, "") registry.registerFunction("CURRENT_USER", info, e => { - if (! e.isEmpty) { + if (e.nonEmpty) { throw new AnalysisException("Argument(s) passed for zero arg function " + s"CURRENT_USER") } diff --git a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala index 9cfd610986..a525611e5a 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/ExistingPlans.scala @@ -30,10 +30,11 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, TableIdentifier} import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, ColumnarStorePartitionedRDD, IndexColumnFormatRelation, SmartConnectorColumnRDD} -import org.apache.spark.sql.execution.columnar.{ColumnTableScan, ConnectionType} +import org.apache.spark.sql.execution.columnar.{ColumnDeleteExec, ColumnPutIntoExec, ColumnTableScan, ColumnUpdateExec, ConnectionType} import org.apache.spark.sql.execution.exchange.{ReusedExchangeExec, ShuffleExchange} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetricInfo, SQLMetrics} import org.apache.spark.sql.execution.row.{RowFormatRelation, RowFormatScanRDD, RowTableScan} +import org.apache.spark.sql.internal.ColumnTableBulkOps import org.apache.spark.sql.sources.{BaseRelation, PrunedUnsafeFilteredScan, SamplingRelation} import org.apache.spark.sql.types._ import org.apache.spark.sql.{AnalysisException, CachedDataFrame, SnappySession} @@ -278,7 +279,8 @@ case class ExecutePlan(child: SparkPlan, preAction: () => Unit = () => ()) SnappySession.getExecutedPlan(child) match { case (u: ColumnUpdateExec, _) => u.connProps case (d: ColumnDeleteExec, _) => d.connProps - case (p: ColumnPutIntoExec, _) => p.updatePlan.asInstanceOf[ColumnUpdateExec].connProps + case (p: ColumnPutIntoExec, _) => + p.updatePlan.asInstanceOf[ColumnUpdateExec].connProps case _ => throw new IllegalStateException( s"Unexpected plan for ${child.getClass.getName}: $child") }, sparkContext) From e7960a9126281d5b0e7765b94cdd492052fbc84a Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Fri, 31 Aug 2018 14:03:58 +0530 Subject: [PATCH 18/19] minor cleanups --- .../java/io/snappydata/impl/SnappyHiveCatalog.java | 2 +- .../SnappyTableStatsProviderService.scala | 5 ++--- .../columnar/impl/JDBCSourceAsColumnarStore.scala | 14 +++++++++++--- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/io/snappydata/impl/SnappyHiveCatalog.java b/core/src/main/java/io/snappydata/impl/SnappyHiveCatalog.java index 18e028ba62..6b18ef0178 100644 --- a/core/src/main/java/io/snappydata/impl/SnappyHiveCatalog.java +++ b/core/src/main/java/io/snappydata/impl/SnappyHiveCatalog.java @@ -465,7 +465,7 @@ private Object invoke() throws Exception { } catch (Exception e) { // ignore exception and move to next Misc.getI18NLogWriter().warning(LocalizedStrings.DEBUG, - "Failed to retrieve information for " + tableName + ": " + e); + "Failed to retrieve information for " + tableName, e); } } } diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index 3d77a51536..69ee0906af 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -216,8 +216,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService new SnappyExternalTableStats(table.entityName, table.tableType, table.shortProvider, table.externalStore, table.dataSourcePath, table.driverClass) } - } - catch { + } catch { case NonFatal(e) => log.warn("Exception occurred while collecting External Table Statistics: " + e.getMessage) log.debug(e.getMessage, e) @@ -237,7 +236,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService val regionStats = result.flatMap(_.getRegionStats.asScala).map(rs => { val tableName = rs.getTableName tableTypesMap.get(tableName.toUpperCase) match { - case Some("COLUMN") => rs.setColumnTable(true) + case Some(t) if t.equalsIgnoreCase("COLUMN") => rs.setColumnTable(true) case _ => rs.setColumnTable(false) } rs diff --git a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala index 3acddd6889..f26b7d059f 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/JDBCSourceAsColumnarStore.scala @@ -36,7 +36,7 @@ import com.pivotal.gemfirexd.internal.iapi.services.context.ContextService import com.pivotal.gemfirexd.internal.impl.jdbc.{EmbedConnection, EmbedConnectionContext} import io.snappydata.impl.SmartConnectorRDDHelper import io.snappydata.thrift.StatementAttrs -import io.snappydata.thrift.internal.{ClientBlob, ClientStatement} +import io.snappydata.thrift.internal.{ClientBlob, ClientConnection, ClientStatement} import org.apache.spark.rdd.RDD import org.apache.spark.serializer.{ConnectionPropertiesSerializer, KryoSerializerPool, StructTypeSerializer} @@ -941,7 +941,16 @@ class SmartConnectorRowRDD(_session: SnappySession, thriftConn.setCommonStatementAttributes(ClientStatement.setLocalExecutionBucketIds( new StatementAttrs(), Collections.singleton(Int.box(bucketPartition.bucketId)), tableName, true).setMetadataVersion(relDestroyVersion).setLockOwner(updateOwner)) + } else thriftConn.setCommonStatementAttributes(null) + try { + executeQuery(thriftConn, conn) + } finally if (isPartitioned) { + thriftConn.setCommonStatementAttributes(null) } + } + + private def executeQuery(thriftConn: ClientConnection, + conn: Connection): (Connection, Statement, ResultSet) = { val sqlText = s"SELECT $columnList FROM ${quotedName(tableName)}$filterWhereClause" val args = filterWhereArgs @@ -964,7 +973,7 @@ class SmartConnectorRowRDD(_session: SnappySession, // get the txid which was used to take the snapshot. if (!commitTx) { - val getSnapshotTXId = conn.prepareStatement("values sys.GET_SNAPSHOT_TXID(?)") + val getSnapshotTXId = thriftConn.prepareStatement("values sys.GET_SNAPSHOT_TXID(?)") getSnapshotTXId.setBoolean(1, delayRollover) val rs = getSnapshotTXId.executeQuery() rs.next() @@ -974,7 +983,6 @@ class SmartConnectorRowRDD(_session: SnappySession, SmartConnectorRDDHelper.snapshotTxIdForRead.set(txId) logDebug(s"The snapshot tx id is $txId and tablename is $tableName") } - thriftConn.setCommonStatementAttributes(null) logDebug(s"The previous snapshot tx id is $txId and tablename is $tableName") (conn, stmt, rs) } From ce7fac90c82f4fc568da396f77e509d857b8f8b1 Mon Sep 17 00:00:00 2001 From: Sumedh Wale Date: Sat, 3 Nov 2018 03:11:45 +0530 Subject: [PATCH 19/19] minor updates to tests --- .../org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala | 2 +- core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala index a78a3aae17..62757d7db5 100644 --- a/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala +++ b/cluster/src/test/scala/org/apache/spark/sql/store/ColumnUpdateDeleteTest.scala @@ -40,7 +40,7 @@ class ColumnUpdateDeleteTest extends ColumnTablesTestBase { stopAll() } - override protected def newSparkConf(addOn: (SparkConf) => SparkConf): SparkConf = { + override protected def newSparkConf(addOn: SparkConf => SparkConf): SparkConf = { val conf = new SparkConf() conf.setIfMissing("spark.master", "local[*]") .setAppName(getClass.getName) diff --git a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index d098c35fd3..7ed0182db8 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -483,7 +483,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { DistributedTestBase.invokeInEveryVM(avoidRollover) avoidRollover.run() - for (_ <- 1 to 10000) { + for (_ <- 1 to 3) { testConcurrentOpsIter(session) session.sql("truncate table updateTable")