diff --git a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala index 772825e647..cb0f67ca5b 100644 --- a/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala +++ b/cluster/src/dunit/scala/io/snappydata/cluster/SplitSnappyClusterDUnitTest.scala @@ -261,13 +261,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) + 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, redundancy = 1) } finally { StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = false } 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 fb77cde5d5..1cca5740d3 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 - && (region.getDataStore != null) - && !region.getAttributes.getEnableOffHeapMemory && !region.isRowBuffer()) + offHeap: Boolean, hasOffHeap: Boolean, onlyRowBuffers: Boolean): Boolean = { + var hasLRU = (region.getEvictionAttributes.getAlgorithm.isLRUHeap + && (region.getDataStore != null) + && !region.getAttributes.getEnableOffHeapMemory) + val isRowBuffer = region.isRowBuffer + 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") @@ -141,7 +149,7 @@ class SnappyStorageEvictor extends Logging { protected def includeLocalRegion(region: LocalRegion): Boolean = { (region.getEvictionAttributes.getAlgorithm.isLRUHeap - && !region.getAttributes.getEnableOffHeapMemory) + && !region.getAttributes.getEnableOffHeapMemory) } } diff --git a/cluster/src/test/scala/io/snappydata/QueryTest.scala b/cluster/src/test/scala/io/snappydata/QueryTest.scala index 838b414d40..c76d29b7b8 100644 --- a/cluster/src/test/scala/io/snappydata/QueryTest.scala +++ b/cluster/src/test/scala/io/snappydata/QueryTest.scala @@ -97,11 +97,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") + } } /** diff --git a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala index cf435524e9..a8a0572897 100644 --- a/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala +++ b/core/src/dunit/scala/io/snappydata/cluster/SplitClusterDUnitTestBase.scala @@ -209,13 +209,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) + 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, redundancy = 1) } finally { StoreUtils.TEST_RANDOM_BUCKETID_ASSIGNMENT = false } diff --git a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala index 1649477e0a..b704b8eeed 100644 --- a/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala +++ b/core/src/main/scala/io/snappydata/SnappyTableStatsProviderService.scala @@ -19,32 +19,41 @@ package io.snappydata -import java.util.concurrent.TimeUnit -import java.util.function.BiFunction +import java.util.Collections +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.function.{BiFunction, Predicate, Function => JFunction} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} import scala.language.implicitConversions import scala.util.control.NonFatal 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.{AbstractRegionEntry, LocalRegion, PartitionedRegion, RegionEntry} +import com.gemstone.gemfire.internal.cache._ 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.Constant._ import io.snappydata.sql.catalog.CatalogObjectType import org.eclipse.collections.impl.map.mutable.UnifiedMap +import org.eclipse.collections.impl.set.mutable.UnifiedSet import org.apache.spark.SparkContext +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.collection.Utils -import org.apache.spark.sql.execution.columnar.impl.{ColumnFormatKey, ColumnFormatRelation, ColumnFormatValue, RemoteEntriesIterator} +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.{SnappyContext, ThinClientConnectorMode} /* @@ -90,8 +99,6 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService if (!doRun) { this.synchronized { if (!doRun) { - val delay = sc.getConf.getLong(PROPERTY_PREFIX + - "calcTableSizeInterval", DEFAULT_CALC_TABLE_SIZE_SERVICE_INTERVAL) doRun = true Misc.getGemFireCache.getCCPTimer.schedule( new SystemTimer.SystemTimerTask { @@ -116,8 +123,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService override def getLoggerI18n: LogWriterI18n = { logger } - }, - delay, delay) + }, delayMillis, delayMillis) } } } @@ -183,7 +189,7 @@ 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 result: Seq[SnappyRegionStatsCollectorResult] = Nil val dataServers = GfxdMessage.getAllDataStores var resultObtained: Boolean = false try { @@ -212,8 +218,7 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService new SnappyExternalTableStats(table.entityName, table.tableType, table.schema, 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) @@ -243,14 +248,10 @@ object SnappyEmbeddedTableStatsProviderService extends TableStatsProviderService }) // Return updated details - (regionStats, - result.flatMap(_.getIndexStats.asScala), - externalTables) + (regionStats, result.flatMap(_.getIndexStats.asScala), externalTables) } else { // Return last successfully updated tableSizeInfo - (tableSizeInfo.values.toSeq, - result.flatMap(_.getIndexStats.asScala), - externalTables) + (tableSizeInfo.values.toSeq, result.flatMap(_.getIndexStats.asScala), externalTables) } } @@ -262,58 +263,283 @@ 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) } } - def publishColumnTableRowCountStats(): Unit = { - def asSerializable[C](c: C) = c.asInstanceOf[C with Serializable] + 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) + } + } + } - val regions = asSerializable(Misc.getGemFireCache.getApplicationRegions.asScala) - for (region: LocalRegion <- regions) { - if (region.getDataPolicy.withPartitioning()) { - val table = Misc.getFullTableNameFromRegionPath(region.getFullPath) + def publishColumnTableRowCountStats(): Unit = { + val cache = Misc.getGemFireCache + 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] - if (ColumnFormatRelation.isColumnTable(table) && - pr.getLocalMaxMemory > 0) { - var numColumnsInTable = -1 + val columnMeta = if (container.isColumnStore) container.fetchHiveMetaData(false) else null + if ((columnMeta ne null) && pr.getLocalMaxMemory > 0) { + 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 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(false /* primaryOnly */ , null /* no TX */ , null /* not required since includeValues is false */ , createRemoteIterator, false /* forUpdate */ , false /* includeValues */) + val maxDeltaRows = pr.getColumnMaxDeltaRows + var smallBucketRegion: BucketRegion = null + val smallBatchBuckets = new UnifiedSet[BucketRegion](2) // using direct region operations while (itr.hasNext) { - val re = itr.next().asInstanceOf[AbstractRegionEntry] - val key = re.getRawKey.asInstanceOf[ColumnFormatKey] - val bucketRegion = itr.getHostedBucketRegion - if (bucketRegion.getBucketAdvisor.isPrimary) { - if (numColumnsInTable < 0) { - numColumnsInTable = key.getNumColumnsInTable(table) + val re = itr.next().asInstanceOf[RegionEntry] + if (!re.isDestroyedOrRemoved) { + val key = re.getRawKey.asInstanceOf[ColumnFormatKey] + val bucketRegion = itr.getHostedBucketRegion + if (bucketRegion.getBucketAdvisor.isPrimary) { + 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) { + if (bucketRegion eq smallBucketRegion) smallBatchBuckets.add(bucketRegion) + else smallBucketRegion = bucketRegion + } + } + re._getValue() match { + case v: ColumnFormatValue => offHeapSize += v.getOffHeapSizeInBytes + case _ => } - rowsInColumnBatch += key.getColumnBatchRowCount(bucketRegion, re, - numColumnsInTable) - } - re._getValue() match { - case v: ColumnFormatValue => offHeapSize += v.getOffHeapSizeInBytes - case _ => } } + itr.close() + // submit a task to merge small batches if required + if (smallBatchBuckets.size() > 0) { + mergeSmallColumnBatches(pr, container, columnMeta, smallBatchBuckets.asScala) + } } val stats = pr.getPrStats stats.setPRNumRowsInColumnBatches(rowsInColumnBatch) stats.setOffHeapSizeInBytes(offHeapSize) + } else if (container.isRowBuffer && pr.getLocalMaxMemory > 0) { + rolloverTasks.computeIfAbsent(pr, rolloverRowBuffersTask) } } } } + + // 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) + + /** + * Check if row buffers are large and have not been touched for a while + * then roll it over into the column table + */ + // noinspection TypeAnnotation + private[this] 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 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: UnifiedSet[BucketRegion] = null + val iter = localPrimaries.iterator() + while (iter.hasNext) { + val br = iter.next() + if (testBucket(br, maxDeltaRows, minModTime) && !br.isLockededForMaintenance) { + if (rolloverBuckets eq null) rolloverBuckets = new UnifiedSet[BucketRegion]() + rolloverBuckets.add(br) + } + } + // 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 ne null) && rolloverBuckets.size() > 0) { + implicit val executionContext = Utils.executionContext(pr.getGemFireCache) + Future { + try { + 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) + } + } + } else null + } else null + } + } + + /** + * 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 + * [[rolloverRowBuffersTask]] or a forced flush of even smaller size for sample tables. + * + * The ColumnBatchIterator is passed the stats row entry. Rest all the columns, including + * delta/delete are looked up by the iterator (see ColumnFormatStatsIterator.getColumnValue) + * when the generated code asks for them. Hence this will be same as iterating batches in + * ColumnTableScan that will return merged entries with deltas/deletes applied. + * The ColumnInsert is tied to output of this hence will create a combined merged batch. + */ + private def mergeSmallColumnBatches(pr: PartitionedRegion, container: GemFireContainer, + metaData: ExternalTableMetaData, smallBatchBuckets: mutable.Set[BucketRegion]): Unit = { + mergeTasks.computeIfAbsent(pr, new JFunction[PartitionedRegion, Future[Unit]] { + override def apply(pr: PartitionedRegion): Future[Unit] = { + val cache = pr.getGemFireCache + implicit val executionContext: ExecutionContext = Utils.executionContext(cache) + Future(Utils.withExceptionHandling({ + val tableName = container.getQualifiedTableName + val schema = Utils.getTableSchema(metaData) + 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) + // zero delta row size to avoid going through rolloverRowBuffers again + val insertPlan = ColumnInsertExec(tableScan, Nil, Nil, + numBuckets = -1, isPartitioned = false, None, + (pr.getColumnBatchSize, 0, 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 + 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}: " + + 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 <- smallBatchBuckets) try { + success = false + locked = false + tx = null + // 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 ${br.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) + } 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.getHostedBucketRegion, + 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 le: LockTimeoutException => logWarning(le.getMessage) + case t: Throwable => Utils.logAndThrowException(t) + } finally { + handleTransaction(cache, tx, context, success) + if (locked) { + br.unlockAfterMaintenance(true, lockOwner) + } + } + }, () => { + mergeTasks.remove(pr) + })) + } + }) + } } diff --git a/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala b/core/src/main/scala/io/snappydata/SnappyThinConnectorTableStatsProvider.scala index f38a6b72c7..2ec1634852 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 c4db2171fb..392c3ca8bd 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.{MemberStatistics, 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, MemberStatistics] = new ConcurrentHashMap[String, MemberStatistics](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) @@ -106,6 +116,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/io/snappydata/ToolsCallback.scala b/core/src/main/scala/io/snappydata/ToolsCallback.scala index 739277ec5c..e5c7939abd 100644 --- a/core/src/main/scala/io/snappydata/ToolsCallback.scala +++ b/core/src/main/scala/io/snappydata/ToolsCallback.scala @@ -69,6 +69,11 @@ trait ToolsCallback { /** * Check permission to write to given schema for a user. Returns the normalized user or * LDAP group name of the schema owner (or passed user itself if security is disabled). + * + * @param schema the schema name for the permission check + * @param currentUser the user for the permission check + * @return if permission is allowed, return the schema owner which can be ldap group + * else this throws an appropriate StandardException */ def checkSchemaPermission(schema: String, currentUser: String): String } diff --git a/core/src/main/scala/io/snappydata/functions.scala b/core/src/main/scala/io/snappydata/functions.scala index cf98e2ae13..03e5630a9e 100644 --- a/core/src/main/scala/io/snappydata/functions.scala +++ b/core/src/main/scala/io/snappydata/functions.scala @@ -39,11 +39,70 @@ import org.apache.spark.unsafe.types.UTF8String */ object SnappyDataFunctions { - val usageStr: String = "_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/io/snappydata/sql/catalog/SmartConnectorHelper.scala b/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala index f2a2faa217..899d4612d9 100644 --- a/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala +++ b/core/src/main/scala/io/snappydata/sql/catalog/SmartConnectorHelper.scala @@ -29,8 +29,8 @@ import com.pivotal.gemfirexd.Attribute import com.pivotal.gemfirexd.internal.engine.distributed.utils.GemFireXDUtils import com.pivotal.gemfirexd.internal.iapi.types.HarmonySerialBlob import com.pivotal.gemfirexd.jdbc.ClientAttribute +import io.snappydata.Constant import io.snappydata.thrift.{BucketOwners, CatalogMetadataDetails, CatalogMetadataRequest} -import io.snappydata.{Constant, Property} import org.eclipse.collections.impl.map.mutable.UnifiedMap import org.apache.spark.sql.SparkSession @@ -227,10 +227,9 @@ object SmartConnectorHelper { if (!buckets.isEmpty) { // check if Spark executors are using IP addresses or host names val preferHost = preferHostName(session) - val preferPrimaries = session.conf.getOption(Property.PreferPrimariesInQuery.name) match { - case None => Property.PreferPrimariesInQuery.defaultValue.get - case Some(p) => p.toBoolean - } + // preferPrimaries is always true here since the information gets cached + // and can be used both for reads and writes (latter should have it as true) + val preferPrimaries = true var orphanBuckets: ArrayBuffer[Int] = null val allNetUrls = new Array[ArrayBuffer[(String, String)]](numBuckets) val availableNetUrls = new UnifiedMap[String, String](4) 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 f70d898e80..d2a892711b 100644 --- a/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala +++ b/core/src/main/scala/org/apache/spark/sql/CachedDataFrame.scala @@ -21,9 +21,8 @@ import java.nio.ByteBuffer 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.concurrent.{Await, ExecutionContext, Future} import scala.reflect.ClassTag import com.esotericsoftware.kryo.io.{Input, Output} @@ -32,6 +31,7 @@ import com.gemstone.gemfire.cache.LowMemoryException import com.gemstone.gemfire.internal.shared.ClientSharedUtils import com.gemstone.gemfire.internal.shared.unsafe.DirectBufferAllocator import com.gemstone.gemfire.internal.{ByteArrayDataInput, ByteBufferDataOutput} +import com.pivotal.gemfirexd.internal.engine.Misc import io.snappydata.Constant import org.apache.spark._ @@ -62,7 +62,7 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti shuffleCleanups: Array[Future[Unit]], val rddId: Int, noSideEffects: Boolean, val queryHints: java.util.Map[String, String], private[sql] var currentExecutionId: Long, private[sql] var planStartTime: Long, private[sql] var planEndTime: Long, - val linkPart : Boolean = false) + val linkPart: Boolean = false) extends Dataset[Row](snappySession, queryExecution, encoder) with Logging { private[sql] final def isCached: Boolean = cachedRDD ne null @@ -117,6 +117,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) => @@ -124,6 +125,7 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti while (i < numShuffleDeps) { val shuffleDependency = shuffleDependencies(i) // Cleaning the shuffle artifacts asynchronously + implicit val executionContext: ExecutionContext = Utils.executionContext(cache) shuffleCleanups(i) = Future { cleaner.doCleanupShuffle(shuffleDependency, blocking = true) } @@ -185,7 +187,7 @@ class CachedDataFrame(snappySession: SnappySession, queryExecution: QueryExecuti private def clearPartitions(rdds: Seq[RDD[_]]): Unit = { val children = rdds.flatMap { case null => Nil - case r => + case r => // f.set(r, null) Platform.putObjectVolatile(r, Utils.rddPartitionsOffset, null) getChildren(r) @@ -441,10 +443,12 @@ final class AggregatePartialDataIterator( object CachedDataFrame extends ((TaskContext, Iterator[InternalRow]) => PartitionResult) - with Serializable with KryoSerializable with Logging { + with Serializable with KryoSerializable with Logging { - @transient @volatile var sparkConf: SparkConf = _ - @transient @volatile var compressionCodec: String = _ + @transient + @volatile var sparkConf: SparkConf = _ + @transient + @volatile var compressionCodec: String = _ override def write(kryo: Kryo, output: Output): Unit = {} diff --git a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala index 61f76e8c6f..4082a08fae 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyContext.scala @@ -1096,18 +1096,17 @@ object SnappyContext extends Logging { val deployCmds = ToolsCallbackInit.toolsCallback.getAllGlobalCmnds val nonEmpty = deployCmds.length > 0 if (nonEmpty) { - logInfo(s"deploycmnds size = ${deployCmds.length}") - deployCmds.foreach(s => logDebug(s"s")) + logInfo(s"Deploy commands size = ${deployCmds.length}") } if (nonEmpty) deployCmds.foreach(d => { + logDebug(s"Deploying: $d") val cmdFields = d.split('|') if (cmdFields.length > 1) { val coordinate = cmdFields(0) val repos = if (cmdFields(1).isEmpty) None else Some(cmdFields(1)) val cache = if (cmdFields(2).isEmpty) None else Some(cmdFields(2)) DeployCommand(coordinate, null, repos, cache, restart = true).run(session) - } - else { + } else { // Jars we have DeployJarCommand(null, cmdFields(0), restart = true).run(session) } 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 7b7d31a990..5d49e5f8f3 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappyDDLParser.scala @@ -648,6 +648,19 @@ abstract class SnappyDDLParser(session: SparkSession) ) } + 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 } 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 d7c8fa3516..5c3d76bebb 100644 --- a/core/src/main/scala/org/apache/spark/sql/SnappySession.scala +++ b/core/src/main/scala/org/apache/spark/sql/SnappySession.scala @@ -230,7 +230,10 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { private[sql] val queryHints = new ConcurrentHashMap[String, String](4, 0.7f, 1) @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 operationContext: Option[OperationContext] = None @transient private[sql] var currentKey: CachedKey = _ @@ -318,8 +321,49 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { } } + 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, new UnifiedMap[Any, Any](4)) + operationContext = Some(context) + context.objects + case Some(context) => context.persist = persist; context.objects + } + // use a unique lock owner + 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) + } + } 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 getMutablePlanTable: String = operationContext match { + case None => null + case Some(context) => + context.objects.get(SnappySession.MUTABLE_PLAN_TABLE).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) + 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 @@ -463,6 +507,10 @@ class SnappySession(_sc: SparkContext) extends SparkSession(_sc) { private[sql] def clearContext(): Unit = synchronized { clearPutInto() contextObjects.clear() + operationContext match { + case Some(context) if !context.persist => operationContext = None + case _ => + } planCaching = Property.PlanCaching.get(sessionState.conf) sqlWarnings = null } @@ -1784,6 +1832,14 @@ 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 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" + + /** a unique UUID of the node for mutability lock ownership */ + private[sql] lazy val MUTABLE_OWNER_PREFIX = java.util.UUID.randomUUID().toString + lazy val isEnterpriseEdition: Boolean = { GemFireCacheImpl.setGFXDSystem(true) GemFireVersion.getInstance(classOf[GemFireXDVersion], SharedUtils.GFXD_VERSION_PROPERTIES) @@ -1881,6 +1937,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() @@ -2241,3 +2298,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: UnifiedMap[Any, Any]) 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 4ec0b01665..15259fabe9 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 @@ -25,13 +25,17 @@ 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 import scala.util.control.NonFatal +import _root_.io.snappydata.sql.catalog.CatalogObjectType import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Kryo, KryoSerializable} +import com.gemstone.gemfire.SystemFailure +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.Misc @@ -49,7 +53,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.{Alias, Attribute, Expression, GenericRow, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Alias, 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} @@ -69,7 +73,7 @@ import org.apache.spark.util.collection.BitSet import org.apache.spark.util.io.ChunkedByteBuffer import org.apache.spark.util.{AccumulatorV2, MutableURLClassLoader} -object Utils { +object Utils extends Logging { final val EMPTY_STRING_ARRAY = Array.empty[String] final val WEIGHTAGE_COLUMN_NAME = "SNAPPY_SAMPLER_WEIGHTAGE" @@ -93,6 +97,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 { @@ -268,10 +299,10 @@ object Utils { private final val timeIntervalSpec = "([0-9]+)(ms|s|m|h)".r /** - * Parse the given time interval value as long milliseconds. - * - * @see timeIntervalSpec for the allowed string specification - */ + * Parse the given time interval value as long milliseconds. + * + * @see timeIntervalSpec for the allowed string specification + */ def parseTimeInterval(optV: Any, module: String): Long = { optV match { case tii: Int => tii.toLong @@ -413,6 +444,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, session: SnappySession): (Boolean, Set[String]) = { if (s.trim.equals("*")) { (true, Set.empty[String]) @@ -465,7 +505,7 @@ object Utils { * field is stored (and rendered) as VARCHAR by SnappyStore. * * @param size the size parameter of the VARCHAR() column type - * @param md optional Metadata object to be merged into the result + * @param md optional Metadata object to be merged into the result * @return the result Metadata object to use for StructField */ def varcharMetadata(size: Int, md: Metadata): Metadata = { @@ -503,7 +543,7 @@ object Utils { * field is stored (and rendered) as CHAR by SnappyStore. * * @param size the size parameter of the CHAR() column type - * @param md optional Metadata object to be merged into the result + * @param md optional Metadata object to be merged into the result * @return the result Metadata object to use for StructField */ def charMetadata(size: Int, md: Metadata): Metadata = { @@ -536,6 +576,18 @@ object Utils { new CaseInsensitiveMutableHashMap[StructField](schema.fields.map(f => f.name -> f).toMap) } + 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 == CatalogObjectType.Sample.toString && + 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) @@ -545,9 +597,9 @@ object Utils { } /** - * Get the result schema given an optional explicit schema and base table. - * In case both are specified, then check compatibility between the two. - */ + * Get the result schema given an optional explicit schema and base table. + * In case both are specified, then check compatibility between the two. + */ def getSchemaAndPlanFromBase(schemaOpt: Option[StructType], baseTableOpt: Option[String], catalog: SnappySessionCatalog, asSelect: Boolean, table: String, @@ -606,8 +658,8 @@ object Utils { } /** - * Register given driver class with Spark's loader. - */ + * Register given driver class with Spark's loader. + */ def registerDriver(driver: String): Unit = { try { DriverRegistry.register(driver) @@ -618,8 +670,8 @@ object Utils { } /** - * Register driver for given JDBC URL and return the driver class name. - */ + * Register driver for given JDBC URL and return the driver class name. + */ def registerDriverUrl(url: String): String = { val driver = getDriverClassName(url) registerDriver(driver) @@ -976,11 +1028,13 @@ final class MultiBucketExecutorPartition(private[this] var _index: Int, private[this] var bucket = bucketSet.nextSetBit(0) override def hasNext: Boolean = bucket >= 0 + override def next(): Integer = { val b = Int.box(bucket) bucket = bucketSet.nextSetBit(bucket + 1) b } + override def remove(): Unit = throw new UnsupportedOperationException } @@ -1055,15 +1109,15 @@ private[spark] case class NarrowExecutorLocalSplitDep( } /** - * Stores information about the narrow dependencies used by a StoreRDD. - * - * @param narrowDep maps to the dependencies variable in the parent RDD: - * for each one to one dependency in dependencies, - * narrowDeps has a NarrowExecutorLocalSplitDep (describing - * the partition for that dependency) at the corresponding - * index. The size of narrowDeps should always be equal to - * the number of parents. - */ + * Stores information about the narrow dependencies used by a StoreRDD. + * + * @param narrowDep maps to the dependencies variable in the parent RDD: + * for each one to one dependency in dependencies, + * narrowDeps has a NarrowExecutorLocalSplitDep (describing + * the partition for that dependency) at the corresponding + * index. The size of narrowDeps should always be equal to + * the number of parents. + */ private[spark] class CoGroupExecutorLocalPartition( idx: Int, val blockId: BlockManagerId, val narrowDep: Option[NarrowExecutorLocalSplitDep]) 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 34c1f8155f..8b49387296 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} @@ -260,7 +261,34 @@ 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/SnappyMetrics.scala b/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala new file mode 100644 index 0000000000..76532a2023 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/sql/execution/SnappyMetrics.scala @@ -0,0 +1,86 @@ +/* + * 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 java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable + +import org.eclipse.collections.impl.list.mutable.primitive.LongArrayList + +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" + + // 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: 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) + 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[LongArrayList]] + val numberFormat = NumberFormat.getIntegerInstance(Locale.US) + valueList.collect { + case l if l ne null => numberFormat.format(l.toArray.sum) + }.mkString("|") + } else SQLMetrics.stringValue(metricType, values.asInstanceOf[LongArrayList].toArray) + } +} 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 04ee9b6092..aa5d530399 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 @@ -142,6 +142,7 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor { }) locations } + inputRDDs.map { rdd => // if the two are different then its partition pruning case if (numBuckets == rdd.getNumPartitions) { @@ -172,3 +173,18 @@ trait TableExec extends UnaryExecNode with CodegenSupportOnExecutor { childProduce } } + +/** + * An iterator that will update provided metrics (those supported by an implementation). + */ +abstract class 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/aggregate/SnappyHashAggregateExec.scala b/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SnappyHashAggregateExec.scala index e6978b6e5d..594e04db9f 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 @@ -465,11 +465,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 8aa252e33a..fac699199c 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,7 +36,9 @@ import org.eclipse.collections.impl.map.mutable.primitive.IntObjectHashMap import org.apache.spark.memory.MemoryManagerCallback.releaseExecutionMemory 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, TaskContextImpl, TaskKilledException} @@ -47,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 @@ -89,34 +91,35 @@ 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 { - if (rs ne null) { - // GfxdConnectionWrapper.restoreContextStack(stmt, rs) - // rs.lightWeightClose() - rs.close() - } - } catch { - case NonFatal(e) => logWarning("Exception closing resultSet", e) + try { + if (rs ne null) { + // GfxdConnectionWrapper.restoreContextStack(stmt, rs) + // rs.lightWeightClose() + rs.close() } - try { - if (stmt ne null) { - stmt.getConnection match { - case embedConn: EmbedConnection => - val lcc = embedConn.getLanguageConnection - if (lcc ne null) { - lcc.clearExecuteLocally() - } - case _ => - } - stmt.close() + } catch { + case NonFatal(e) => logWarning("Exception closing resultSet", e) + } + try { + if (stmt ne null) { + stmt.getConnection match { + case embedConn: EmbedConnection => + val lcc = embedConn.getLanguageConnection + if (lcc ne null) { + lcc.clearExecuteLocally() + } + case _ => } - } catch { - case NonFatal(e) => logWarning("Exception closing statement", e) + stmt.close() } - hasNextValue = false - + } catch { + case NonFatal(e) => logWarning("Exception closing statement", e) + } + hasNextValue = false } } @@ -125,18 +128,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, context) { if (region ne null) { @@ -150,11 +161,14 @@ 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 = _ + private var remoteBatchesMetric: 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 @@ -162,14 +176,21 @@ 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) + 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] @@ -177,18 +198,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 @@ -198,7 +218,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 } @@ -269,6 +289,7 @@ final class ColumnBatchIterator(region: LocalRegion, val 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 @@ -286,8 +307,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 @@ -309,6 +329,13 @@ final class ColumnBatchIterator(region: LocalRegion, val batch: ColumnBatch, } } + override def setMetric(name: String, metric: SQLMetric): Boolean = name match { + 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 + } + 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 638be95d64..1d8f22fd7d 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 @@ -98,7 +98,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 b4fb6a2362..29a91576bf 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 @@ -76,9 +76,8 @@ 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 batchBucketIdTerm: Option[String] = None + @transient private[this] var batchIdRef: Int = -1 + @transient private[this] var batchBucketIdTerm: Option[String] = None def columnBatchSize: Int = batchParams._1 @@ -90,13 +89,15 @@ 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 if (batchSize >= 16 * 1024 * 1024) ("16", "0x0f") - else if (batchSize >= 8 * 1024 * 1024) ("8", "0x07") - else if (batchSize >= 4 * 1024 * 1024) ("4", "0x03") - else if (batchSize >= 2 * 1024 * 1024) ("2", "0x01") + else if (batchSize >= 8 * 1024 * 1024) ("8", "0x07") + else if (batchSize >= 4 * 1024 * 1024) ("4", "0x03") + else if (batchSize >= 2 * 1024 * 1024) ("2", "0x01") else ("1", "0x0") } 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 c24b6efb23..213a2f6a4a 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,6 +47,7 @@ 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.encoding._ import org.apache.spark.sql.execution.columnar.impl.{BaseColumnFormatRelation, ColumnDelta} @@ -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]], @@ -107,20 +108,33 @@ 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"), - "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 = @@ -183,10 +197,14 @@ 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 numBatchesRemote = metricTerm(ctx, NUM_BATCHES_REMOTE) 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 @@ -220,6 +238,7 @@ private[sql] 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 if (otherRDDs.isEmpty) { @@ -330,6 +349,22 @@ 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""" + $colInput.setMetric("$NUM_BATCHES_DISK_PARTIAL", $numBatchesDiskPartial); + $colInput.setMetric("$NUM_BATCHES_DISK_FULL", $numBatchesDiskFull); + $colInput.setMetric("$NUM_BATCHES_REMOTE", $numBatchesRemote); + """ + } val initRowTableDecoders = new StringBuilder val bufferInitCodeBlocks = new ArrayBuffer[String]() @@ -497,7 +532,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) { @@ -640,6 +676,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 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 5dc0cd8aec..d5996d62f9 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 @@ -656,11 +656,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/encoding/DictionaryEncoding.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala index ad828b8d96..e0337d6822 100644 --- a/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala +++ b/core/src/main/scala/org/apache/spark/sql/execution/columnar/encoding/DictionaryEncoding.scala @@ -20,8 +20,8 @@ import java.nio.ByteBuffer import com.gemstone.gemfire.internal.cache.GemFireCacheImpl import com.gemstone.gemfire.internal.shared.BufferAllocator -import com.gemstone.gnu.trove.TLongArrayList import io.snappydata.collection.{DictionaryMap, LongKey, ObjectHashSet} +import org.eclipse.collections.impl.list.mutable.primitive.LongArrayList import org.apache.spark.sql.collection.Utils import org.apache.spark.sql.types._ @@ -177,7 +177,7 @@ trait DictionaryEncoderBase extends ColumnEncoder with DictionaryEncoding { private final var stringMap: DictionaryMap = _ private final var longMap: ObjectHashSet[LongIndexKey] = _ - private final var longArray: TLongArrayList = _ + private final var longArray: LongArrayList = _ private final var isIntMap: Boolean = _ private final var writeHeader: Boolean = true @@ -232,7 +232,7 @@ trait DictionaryEncoderBase extends ColumnEncoder with DictionaryEncoding { val mapSize = if (longMap ne null) longMap.size else math.min(math.max(initSize >>> 1, 128), 1024) longMap = new ObjectHashSet[LongIndexKey](mapSize, 0.6, 1, false) - longArray = new TLongArrayList(mapSize) + longArray = new LongArrayList(mapSize) isIntMap = t.isInstanceOf[IntegerType] } if (withHeader) initializeLimits() @@ -364,7 +364,7 @@ trait DictionaryEncoderBase extends ColumnEncoder with DictionaryEncoding { } else if (isIntMap) { var index = 0 while (index < numDictionaryElements) { - val l = longArray.getQuick(index) + val l = longArray.get(index) ColumnEncoding.writeInt(columnBytes, cursor, l.toInt) cursor += 4 index += 1 @@ -372,7 +372,7 @@ trait DictionaryEncoderBase extends ColumnEncoder with DictionaryEncoding { } else { var index = 0 while (index < numDictionaryElements) { - val l = longArray.getQuick(index) + val l = longArray.get(index) ColumnEncoding.writeLong(columnBytes, cursor, l) cursor += 8 index += 1 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/ColumnDelta.scala b/core/src/main/scala/org/apache/spark/sql/execution/columnar/impl/ColumnDelta.scala index 7003f57f73..7fbf4cb7b5 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 = 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) @@ -300,8 +297,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 = { @@ -312,7 +308,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 0ac2813b75..20e40d0ef8 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,6 +30,7 @@ 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 @@ -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 = 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 53659398aa..a349e977a9 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 @@ -32,8 +32,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 @@ -104,19 +103,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 = + Utils.getTableSchema(columnTable.fetchHiveMetaData(false)).length override def getColumnBatchRowCount(bucketRegion: BucketRegion, - re: AbstractRegionEntry, numColumnsInTable: Int): Int = { - val currentBucketRegion = bucketRegion.getHostedBucketRegion - if ((columnIndex == ColumnFormatEntry.STATROW_COL_INDEX || + re: RegionEntry, numColumnsInTable: Int): Int = { + if (columnIndex == ColumnFormatEntry.STATROW_COL_INDEX || columnIndex == ColumnFormatEntry.DELTA_STATROW_COL_INDEX || - columnIndex == ColumnFormatEntry.DELETE_MASK_COL_INDEX) && - !re.isDestroyedOrRemoved) { - val statsOrDeleteVal = re.getValue(currentBucketRegion) + columnIndex == ColumnFormatEntry.DELETE_MASK_COL_INDEX) { + val statsOrDeleteVal = re.getValue(bucketRegion) if (statsOrDeleteVal ne null) { val statsOrDelete = statsOrDeleteVal.asInstanceOf[ColumnFormatValue] .getValueRetain(FetchRequest.DECOMPRESS) @@ -727,8 +722,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) { @@ -737,6 +733,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 fa44601a11..9c4a7b87ff 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 org.eclipse.collections.impl.map.mutable.primitive.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 = new LongObjectHashMapWithState[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,28 @@ 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.forEachValue(new Procedure[AnyRef] { + override def value(v: AnyRef): Unit = v match { + case s: SerializedDiskBuffer => s.release() + case _ => + } + }) + entryMap.clear() } } @@ -317,21 +334,22 @@ private final class DiskMultiColumnBatch(_statsEntry: RegionEntry, _region: Loca 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: LongObjectHashMapWithState[AnyRef] = { - if (closing) null - else { + private[impl] def fillEntryMap(map: LongObjectHashMapWithState[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 = new LongObjectHashMapWithState[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 { @@ -344,7 +362,11 @@ private final class DiskMultiColumnBatch(_statsEntry: RegionEntry, _region: Loca 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) + } } } @@ -370,13 +392,14 @@ private final class DiskMultiColumnBatch(_statsEntry: RegionEntry, _region: Loca } 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) @@ -389,25 +412,50 @@ private final class DiskMultiColumnBatch(_statsEntry: RegionEntry, _region: Loca 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.forEachValue(new Procedure[AnyRef] { - override def value(v: AnyRef): Unit = { - v match { - case s: SerializedDiskBuffer => s.release() - case _ => - } - } - }) - entryMap.clear() - } +/** + * 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 ClusteredDiskIterator with DiskRegionIterator { + + 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 initDiskIterator(): Boolean = false + + override def setRegion(region: LocalRegion): Unit = {} + + override def close(): Unit = currentKey = null } final class LongObjectHashMapWithState[V](expectedSize: 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 b730edec6d..0ac0e36524 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 @@ -246,6 +246,11 @@ abstract class BaseColumnFormatRelation( partitioningColumns ++ ColumnDelta.mutableKeyNames } + protected def markMutablePlan(): Unit = { + sqlContext.sparkSession.asInstanceOf[SnappySession].setMutablePlanOwner( + resolvedName, persist = false) + } + /** Get key columns of the column table */ override def getPrimaryKeyColumns: Seq[String] = { val keyColsOptions = origOptions.get(ExternalStoreUtils.KEY_COLUMNS) @@ -261,6 +266,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) @@ -272,6 +278,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) @@ -357,9 +364,9 @@ 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. */ - private def createExternalTableForColumnBatches(tableName: String, conn: Connection): Unit = { + private def createTableForColumnBatches(tableName: String, conn: Connection): 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 @@ -406,11 +413,9 @@ abstract class BaseColumnFormatRelation( // setting table created to true here as cleanup // in case of failed creation does a exists check. tableCreated = true - dialect match { - case d: JdbcExtendedDialect => d.initializeTable(resolvedName, - sqlContext.conf.caseSensitiveAnalysis, conn) - } - createExternalTableForColumnBatches(externalColumnTableName, conn) + dialect.asInstanceOf[JdbcExtendedDialect].initializeTable(resolvedName, + sqlContext.conf.caseSensitiveAnalysis, conn) + createTableForColumnBatches(externalColumnTableName, conn) // store schema will miss complex types etc, so use the user-provided one val session = sqlContext.sparkSession.asInstanceOf[SnappySession] session.externalCatalog.invalidate(schemaName -> tableName) @@ -601,6 +606,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) } @@ -695,7 +701,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 619a17cc28..5b2e5011de 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 @@ -37,7 +37,7 @@ import com.pivotal.gemfirexd.internal.impl.jdbc.{EmbedConnection, EmbedConnectio import io.snappydata.impl.SmartConnectorRDDHelper import io.snappydata.sql.catalog.SmartConnectorHelper import io.snappydata.thrift.StatementAttrs -import io.snappydata.thrift.internal.{ClientBlob, ClientPreparedStatement, ClientStatement} +import io.snappydata.thrift.internal.{ClientBlob, ClientConnection, ClientStatement} import org.apache.spark.rdd.RDD import org.apache.spark.serializer.{ConnectionPropertiesSerializer, KryoSerializerPool, StructTypeSerializer} @@ -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) { @@ -913,10 +919,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) } @@ -925,34 +931,30 @@ 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).setCatalogVersion(catalogSchemaVersion)) - } else { - clientConn.setCommonStatementAttributes( - new StatementAttrs().setCatalogVersion(catalogSchemaVersion)) - } - clientConn - case _ => null + 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).setCatalogVersion(catalogSchemaVersion).setLockOwner(updateOwner)) + } else { + thriftConn.setCommonStatementAttributes( + new StatementAttrs().setCatalogVersion(catalogSchemaVersion)) } - if (isPartitioned && (thriftConn eq null)) { - val ps = conn.prepareStatement("call sys.SET_BUCKETS_FOR_LOCAL_EXECUTION(?, ?, ?)") - ps.setString(1, tableName) - val bucketString = bucketPartition.bucketId.toString - ps.setString(2, bucketString) - ps.setLong(3, catalogSchemaVersion) - ps.executeUpdate() - ps.close() + try { + executeQuery(thriftConn, conn, txId) + } finally if (isPartitioned) { + thriftConn.setCommonStatementAttributes(null) } + } + + private def executeQuery(thriftConn: ClientConnection, + conn: Connection, txId: String): (Connection, Statement, ResultSet) = { 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) } @@ -960,21 +962,17 @@ class SmartConnectorRowRDD(_session: SnappySession, if (fetchSize ne null) { stmt.setFetchSize(fetchSize.toInt) } + stmt.setSnapshotTransactionId(txId) - 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')") - } - } - + // TODO: 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. if (!commitTx) { - val getTXIdAndHostUrl = conn.prepareStatement("values sys.GET_SNAPSHOT_TXID_AND_HOSTURL(?)") + val getTXIdAndHostUrl = thriftConn.prepareStatement( + "values sys.GET_SNAPSHOT_TXID_AND_HOSTURL(?)") getTXIdAndHostUrl.setBoolean(1, delayRollover) val rs = getTXIdAndHostUrl.executeQuery() rs.next() @@ -985,9 +983,6 @@ class SmartConnectorRowRDD(_session: SnappySession, SmartConnectorHelper.snapshotTxIdForRead.set(txIdAndHostUrl) logDebug(s"The snapshot tx id is $txIdAndHostUrl and tablename is $tableName") } - if (thriftConn ne null) { - thriftConn.setCommonStatementAttributes(null) - } logDebug(s"The previous snapshot tx id is $txId and tablename is $tableName") (conn, stmt, rs) } 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 ecf8255933..1716d0dd4a 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 org.eclipse.collections.api.block.procedure.Procedure import org.eclipse.collections.impl.map.mutable.primitive.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 = new IntObjectHashMap[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) @@ -208,6 +216,7 @@ final class RemoteEntriesIterator(bucketId: Int, projection: Array[Int], fetchUsingGetAll(fetchKeys).foreach { case (k: ColumnFormatKey, v) => currentValueMap.put(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 d1ac23c1e1..5ba2f0dc97 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 @@ -30,6 +30,7 @@ import com.gemstone.gemfire.internal.cache.{BucketRegion, EntryEventImpl, Extern import com.gemstone.gemfire.internal.shared.{FetchRequest, SystemProperties} 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 @@ -126,15 +127,9 @@ 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 == CatalogObjectType.Sample.toString && - 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, tableName, - ColumnFormatRelation.columnBatchTableName(tableName), schema, + ColumnFormatRelation.columnBatchTableName(tableName, None), schema, catalogEntry.externalStore.asInstanceOf[ExternalStore], catalogEntry.compressionCodec) batchCreator.createAndStoreBatch(sc, row, @@ -207,7 +202,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 @@ -535,6 +530,14 @@ object StoreCallbacksImpl extends StoreCallbacks with Logging with Serializable 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() + } + override def getLeadClassLoader: URLClassLoader = ToolsCallbackInit.toolsCallback.getLeadClassLoader 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 d3d10eb369..36eda4129e 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._ @@ -42,8 +42,9 @@ 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.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, SnappyMetrics} import org.apache.spark.sql.sources.JdbcExtendedUtils.quotedName import org.apache.spark.sql.sources._ import org.apache.spark.{Partition, TaskContext, TaskContextImpl, TaskKilledException} @@ -66,6 +67,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, @transient protected val region: Option[LocalRegion]) extends RDDKryo[Any](session.sparkContext, Nil) with KryoSerializable { + protected var updateOwner: String = _ protected var filterWhereArgs: ArrayBuffer[Any] = _ /** * `filters`, but as a WHERE clause suitable for injection into a SQL query. @@ -73,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 ") @@ -185,11 +190,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) } @@ -198,7 +203,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 { @@ -207,6 +212,7 @@ class RowFormatScanRDD(@transient val session: SnappySession, } ps.setString(2, bucketString) ps.setLong(3, -1) + if (updateOwner ne null) ps.setString(4, updateOwner) else ps.setNull(4, Types.VARCHAR) ps.executeUpdate() } finally { ps.close() @@ -270,6 +276,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) { @@ -279,6 +290,12 @@ class RowFormatScanRDD(@transient val session: SnappySession, val txManagerImpl = GemFireCacheImpl.getExisting.getCacheTransactionManager var tx = txManagerImpl.getTXState val startTX = tx eq null + // acquire bucket maintenance read lock if required before snapshot gets acquired + container.getRegion match { + case pr: PartitionedRegion if updateOwner ne null => + GfxdSystemProcedures.lockPrimaryForMaintenance(false, updateOwner, pr, bucketIds) + case _ => + } if (startTX) { tx = txManagerImpl.beginTX(TXManagerImpl.getOrCreateTXContext, IsolationLevel.SNAPSHOT, null, null) @@ -286,21 +303,17 @@ 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 container = GemFireXDUtils.getGemFireContainer(tableName, true) - val bucketIds = thePart match { - case p: MultiBucketExecutorPartition => p.buckets - case _ => java.util.Collections.singleton(Int.box(thePart.index)) - } - + val itr: IteratorWithMetrics[_] = if (isPartitioned && filterWhereClause.isEmpty) { val txId = if (tx ne null) tx.getTransactionId else null - val itr = new CompactExecRowIteratorOnScan(container, bucketIds, txId, context) + // always fault-in for row buffers + val itr = new CompactExecRowIteratorOnScan(container, bucketIds, txId, + faultIn = container.isRowBuffer, context) if (useResultSet) { // row buffer of column table: wrap a result set around the scan 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) @@ -371,6 +384,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) @@ -404,6 +418,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) @@ -417,13 +432,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) + stmt: Statement, val rs: ResultSet, context: TaskContext, + source: Option[IteratorWithMetrics[_]] = None) extends ResultSetIterator[Void](conn, stmt, rs, context) { 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, @@ -437,7 +458,7 @@ final class CompactExecRowIteratorOnRS(conn: Connection, } abstract class PRValuesIterator[T](container: GemFireContainer, region: LocalRegion, - bucketIds: java.util.Set[Integer], context: TaskContext) extends Iterator[T] { + bucketIds: java.util.Set[Integer], context: TaskContext) extends IteratorWithMetrics[T] { protected type PRIterator = PartitionedRegion#PRLocalScanIterator @@ -450,12 +471,10 @@ abstract class PRValuesIterator[T](container: GemFireContainer, region: LocalReg 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 @@ -489,24 +508,41 @@ abstract class PRValuesIterator[T](container: GemFireContainer, region: LocalReg } final class CompactExecRowIteratorOnScan(container: GemFireContainer, - bucketIds: java.util.Set[Integer], txId: TXId, context: TaskContext) + bucketIds: java.util.Set[Integer], txId: TXId, faultIn: Boolean, context: TaskContext) extends PRValuesIterator[AbstractCompactExecRow](container, region = null, bucketIds, context) { 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 + 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) + 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 == 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 b91089b25a..568038a77a 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,24 +16,31 @@ */ package org.apache.spark.sql.execution.ui +import java.util.AbstractMap.SimpleEntry + +import scala.collection.JavaConverters._ import scala.collection.mutable +import org.eclipse.collections.api.block.function.{Function0 => JFunction} +import org.eclipse.collections.impl.list.mutable.primitive.LongArrayList +import org.eclipse.collections.impl.map.mutable.UnifiedMap + 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} /** * A new event that is fired when a plan is executed to get an RDD. */ case class SparkListenerSQLPlanExecutionStart( - executionId: Long, - description: String, - details: String, - physicalPlanDescription: String, - sparkPlanInfo: SparkPlanInfo, - time: Long) - extends SparkListenerEvent + executionId: Long, + description: String, + details: String, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo, + time: Long) + extends SparkListenerEvent /** * Snappy's SQL Listener. @@ -44,18 +51,18 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { // base class variables that are private private val baseStageIdToStageMetrics = { getInternalField("org$apache$spark$sql$execution$ui$SQLListener$$_stageIdToStageMetrics"). - asInstanceOf[mutable.HashMap[Long, SQLStageMetrics]] + asInstanceOf[mutable.HashMap[Long, SQLStageMetrics]] } private val baseJobIdToExecutionId = { getInternalField("org$apache$spark$sql$execution$ui$SQLListener$$_jobIdToExecutionId"). - asInstanceOf[mutable.HashMap[Long, Long]] + asInstanceOf[mutable.HashMap[Long, Long]] } private val baseActiveExecutions = { getInternalField("activeExecutions").asInstanceOf[mutable.HashMap[Long, SQLExecutionUIData]] } private val baseExecutionIdToData = { getInternalField("org$apache$spark$sql$execution$ui$SQLListener$$_executionIdToData"). - asInstanceOf[mutable.HashMap[Long, SQLExecutionUIData]] + asInstanceOf[mutable.HashMap[Long, SQLExecutionUIData]] } def getInternalField(fieldName: String): Any = { @@ -80,7 +87,7 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { // in the active executions. For such cases, we need to // look up the executionUIToData as well. val executionData = baseActiveExecutions.get(executionId). - orElse(baseExecutionIdToData.get(executionId)) + orElse(baseExecutionIdToData.get(executionId)) executionData.foreach { executionUIData => executionUIData.jobs(jobId) = JobExecutionStatus.RUNNING executionUIData.stages ++= stageIds @@ -111,23 +118,23 @@ class SnappySQLListener(conf: SparkConf) extends SQLListener(conf) { case SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) => synchronized { val executionUIData = baseExecutionIdToData.getOrElseUpdate(executionId, { - val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) - val sqlPlanMetrics = physicalPlanGraph.allNodes.flatMap { node => - node.metrics.map(metric => metric.accumulatorId -> metric) - } - // description and details strings being reference equals means - // trim off former here - val desc = if (description eq details) { - CachedDataFrame.queryStringShortForm(details) - } else description - new SQLExecutionUIData( - executionId, - desc, - details, - physicalPlanDescription, - physicalPlanGraph, - sqlPlanMetrics.toMap, - time) + val physicalPlanGraph = SparkPlanGraph(sparkPlanInfo) + val sqlPlanMetrics = physicalPlanGraph.allNodes.flatMap { node => + node.metrics.map(metric => metric.accumulatorId -> metric) + } + // description and details strings being reference equals means + // trim off former here + val desc = if (description eq details) { + CachedDataFrame.queryStringShortForm(details) + } else description + new SQLExecutionUIData( + executionId, + desc, + details, + physicalPlanDescription, + physicalPlanGraph, + sqlPlanMetrics.toMap, + time) }) baseActiveExecutions(executionId) = executionUIData } @@ -150,6 +157,85 @@ 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 expandBuffer(b: mutable.ArrayBuffer[LongArrayList], 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] = { + // 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 = new UnifiedMap[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 + val mapValue = accumulatorMap.getIfAbsentPut(key, new JFunction[MapValue] { + override def value(): MapValue = + new MapValue(new mutable.ArrayBuffer[LongArrayList](math.max(index + 1, 4)), 0L) + }) + val valueList = mapValue.getKey.asInstanceOf[mutable.ArrayBuffer[LongArrayList]] + expandBuffer(valueList, index + 1) + val values = valueList(index) match { + case null => val l = new LongArrayList(4); valueList(index) = l; l + case l => l + } + values.add(value.asInstanceOf[Long]) + if (index == 0) mapValue.setValue(accumulatorId) + } else { + val mapValue = accumulatorMap.getIfAbsentPut(accumulatorId, new JFunction[MapValue] { + override def value(): MapValue = new MapValue(new LongArrayList(4), metricType) + }) + mapValue.getKey.asInstanceOf[LongArrayList].add(value.asInstanceOf[Long]) + } + } + // 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.asInstanceOf[java.util.Map[Any, MapValue]].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/internal/ColumnTableBulkOps.scala b/core/src/main/scala/org/apache/spark/sql/internal/ColumnTableBulkOps.scala index 85943e69ef..9b57dd0602 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,19 +16,22 @@ */ 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.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, AttributeSet, 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, StructType} -import org.apache.spark.sql.{AnalysisException, Dataset, Row, SnappySession, SparkSession} +import org.apache.spark.sql.{AnalysisException, Dataset, Row, SnappyContext, SnappySession, SparkSession, ThinClientConnectorMode} /** * Helper object for PutInto operations for column tables. @@ -42,9 +45,12 @@ object ColumnTableBulkOps { val table = originalPlan.table val subQuery = originalPlan.child var transFormedPlan: LogicalPlan = originalPlan + val session = sparkSession.asInstanceOf[SnappySession] + var success = false + var tableName: String = null table.collectFirst { - case LogicalRelation(mutable: BulkPutRelation, _, _) => + case LogicalRelation(mutable: BulkPutRelation, _, _) => try { val putKeys = mutable.getPutKeys match { case None => throw new AnalysisException( s"PutInto in a column table requires key column(s) but got empty string") @@ -52,7 +58,8 @@ object ColumnTableBulkOps { } val condition = prepareCondition(sparkSession, table, subQuery, putKeys) - val 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)) @@ -66,6 +73,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, Nil, updateColumns, updateExpressions) val updateDS = new Dataset(sparkSession, updatePlan, RowEncoder(updatePlan.schema)) @@ -94,11 +104,54 @@ 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) { + 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.resolveRelation(session.tableIdentifier(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) => @@ -143,9 +196,10 @@ object ColumnTableBulkOps { newCondition } - def getKeyColumns(table: LogicalPlan): Set[String] = { + def getKeyColumns(table: LogicalPlan): (String, Set[String]) = { table.collectFirst { - case LogicalRelation(mutable: MutableRelation, _, _) => mutable.getKeyColumns.toSet + case LogicalRelation(mutable: MutableRelation, _, _) => + mutable.table -> mutable.getKeyColumns.toSet } match { case None => throw new AnalysisException( s"Update/Delete requires a MutableRelation but got $table") 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 d6ddfd57bb..cad4bec587 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 @@ -43,8 +43,8 @@ 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. 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 3e5b19994a..4360f5ff26 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 @@ -25,6 +25,7 @@ import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedM import com.gemstone.gemfire.internal.cache.{CacheDistributionAdvisee, PartitionedRegion} import com.pivotal.gemfirexd.internal.engine.{GfxdConstants, Misc} import io.snappydata.sql.catalog.SnappyExternalCatalog +import org.eclipse.collections.api.block.function.{Function => JFunction} import org.eclipse.collections.impl.map.mutable.UnifiedMap import org.apache.spark.Partition @@ -115,6 +116,15 @@ object StoreUtils { // which is enabled per-query using `LinkPartitionsToBuckets` rule private[sql] val PROPERTY_PARTITION_BUCKET_LINKED = "linkPartitionsToBuckets" + private type ServerBucket = (Option[BlockAndExecutorId], mutable.ArrayBuffer[Int]) + + private[this] val initBucketList: JFunction[Option[BlockAndExecutorId], ServerBucket] = + new JFunction[Option[BlockAndExecutorId], ServerBucket] { + override def valueOf(blockId: Option[BlockAndExecutorId]): ServerBucket = { + blockId -> new mutable.ArrayBuffer[Int]() + } + } + def lookupName(tableName: String, schema: String): String = { val lookupName = { if (tableName.indexOf('.') <= 0) { @@ -220,8 +230,7 @@ object StoreUtils { region: PartitionedRegion, preferPrimaries: Boolean): Array[Partition] = { val numTotalBuckets = region.getTotalNumberOfBuckets - val serverToBuckets = new UnifiedMap[InternalDistributedMember, - (Option[BlockAndExecutorId], mutable.ArrayBuffer[Int])](4) + val serverToBuckets = new UnifiedMap[InternalDistributedMember, ServerBucket](4) val adviser = region.getRegionAdvisor for (p <- 0 until numTotalBuckets) { var prefNode = if (preferPrimaries) region.getOrCreateNodeForBucketWrite(p, null) @@ -236,22 +245,19 @@ object StoreUtils { adviser.getBucketOwners(p).asScala.collectFirst( new PartialFunction[InternalDistributedMember, BlockAndExecutorId] { private var b: Option[BlockAndExecutorId] = None + override def isDefinedAt(m: InternalDistributedMember): Boolean = { b = SnappyContext.getBlockId(m.canonicalString()) b.isDefined } + override def apply(m: InternalDistributedMember): BlockAndExecutorId = { - prefNode = m; b.get + prefNode = m + b.get } }) } - 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.getIfAbsentPutWith(prefNode, initBucketList, prefBlockId)._2 buckets += p } // marker array to check that all buckets have been allocated @@ -454,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}' ") @@ -468,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/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala index 2e44e4a15d..a9fed5449a 100644 --- a/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala +++ b/core/src/test/scala/io/snappydata/ColumnUpdateDeleteTests.scala @@ -17,7 +17,7 @@ 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 @@ -38,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") @@ -49,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))", @@ -199,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") @@ -246,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") @@ -255,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) @@ -359,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") @@ -412,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") @@ -441,7 +449,7 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { session.conf.unset(Property.ColumnBatchSize.name) } - def testConcurrentOps(session: SnappySession): Unit = { + 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") @@ -451,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() { @@ -512,8 +521,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) @@ -546,6 +555,12 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { var res = session.sql( "select * from updateTable EXCEPT select * from checkTable1").collect() + if (res.length != 0) { // SW: + // scalastyle:off println + println("Failed in updates?") + // scalastyle:on println + Thread.sleep(10000000) + } assert(res.length === 0) // concurrent deletes @@ -575,17 +590,29 @@ object ColumnUpdateDeleteTests extends Assertions with Logging { res = session.sql( "select * from updateTable EXCEPT select * from checkTable2").collect() + if (res.length != 0) { // SW: + // scalastyle:off println + println("Failed in deletes?") + // scalastyle:on println + Thread.sleep(10000000) + } assert(res.length === 0) + + executorService.shutdown() + if (!executorService.awaitTermination(1, TimeUnit.SECONDS)) { + executorService.shutdownNow() + } } - def testSNAP2124(session: SnappySession): Unit = { + def testSNAP2124(session: SnappySession, 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/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala b/core/src/test/scala/org/apache/spark/sql/store/ColumnTableInternalValidationTest.scala index 49904a49be..b829e0414f 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/store b/store index 6a0d8628d1..644e110187 160000 --- a/store +++ b/store @@ -1 +1 @@ -Subproject commit 6a0d8628d16eef4759258f2cf3e7fa73350ffd2f +Subproject commit 644e110187a00350a96ddb82d15d0968cc01ca52