Return-Path: X-Original-To: apmail-phoenix-commits-archive@minotaur.apache.org Delivered-To: apmail-phoenix-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 187F118B47 for ; Wed, 24 Feb 2016 01:11:15 +0000 (UTC) Received: (qmail 40598 invoked by uid 500); 24 Feb 2016 01:11:02 -0000 Delivered-To: apmail-phoenix-commits-archive@phoenix.apache.org Received: (qmail 40550 invoked by uid 500); 24 Feb 2016 01:11:02 -0000 Mailing-List: contact commits-help@phoenix.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@phoenix.apache.org Delivered-To: mailing list commits@phoenix.apache.org Received: (qmail 40537 invoked by uid 99); 24 Feb 2016 01:11:02 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 24 Feb 2016 01:11:02 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2FCB2E8E7C; Wed, 24 Feb 2016 01:11:02 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jamestaylor@apache.org To: commits@phoenix.apache.org Date: Wed, 24 Feb 2016 01:11:02 -0000 Message-Id: <5f224af5a24f41b7acf58e5034e7b337@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] phoenix git commit: Revert "PHOENIX-2706 Implement client-side mechanism to know if stats are enabled" Repository: phoenix Updated Branches: refs/heads/4.x-HBase-1.0 f3e5831b2 -> 7e40b89f2 Revert "PHOENIX-2706 Implement client-side mechanism to know if stats are enabled" This reverts commit 534bd8ef6e64970fb6d9b3a65e34adc7535c9204. Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/e00550c7 Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/e00550c7 Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/e00550c7 Branch: refs/heads/4.x-HBase-1.0 Commit: e00550c71123519e15068af27a929dcb56b45922 Parents: f3e5831 Author: James Taylor Authored: Tue Feb 23 17:05:43 2016 -0800 Committer: James Taylor Committed: Tue Feb 23 17:05:43 2016 -0800 ---------------------------------------------------------------------- .../StatisticsCollectionRunTrackerIT.java | 11 ++-- .../org/apache/phoenix/end2end/IndexToolIT.java | 3 +- .../phoenix/end2end/MutableIndexToolIT.java | 1 - .../end2end/StatsCollectionDisabledIT.java | 19 ++----- .../phoenix/end2end/StatsCollectorIT.java | 3 -- .../coprocessor/MetaDataEndpointImpl.java | 12 ++++- .../UngroupedAggregateRegionObserver.java | 27 +++++----- .../phoenix/iterate/BaseResultIterators.java | 25 +-------- .../phoenix/query/ConnectionQueryServices.java | 3 -- .../query/ConnectionQueryServicesImpl.java | 25 ++------- .../query/ConnectionlessQueryServicesImpl.java | 18 +------ .../query/DelegateConnectionQueryServices.java | 10 ---- .../org/apache/phoenix/query/QueryServices.java | 1 - .../phoenix/query/QueryServicesOptions.java | 7 --- .../apache/phoenix/schema/MetaDataClient.java | 2 - .../stats/StatisticsCollectionRunTracker.java | 7 +-- .../phoenix/schema/stats/StatisticsUtil.java | 16 ------ .../org/apache/phoenix/util/MetaDataUtil.java | 55 +++++++------------- .../phoenix/query/QueryServicesTestImpl.java | 2 - .../apache/phoenix/util/MetaDataUtilTest.java | 33 ------------ 20 files changed, 66 insertions(+), 214 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java index 18aeb6c..c64038e 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/coprocessor/StatisticsCollectionRunTrackerIT.java @@ -17,7 +17,8 @@ */ package org.apache.phoenix.coprocessor; -import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.UPDATE_STATS_SKIPPED; +import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT; +import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -36,6 +37,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker; import org.apache.phoenix.util.ReadOnlyProps; +import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; @@ -100,8 +102,7 @@ public class StatisticsCollectionRunTrackerIT extends BaseOwnClusterHBaseManaged HRegionInfo regionInfo = createTableAndGetRegion(tableName); // simulate stats collection via major compaction by marking the region as compacting in the tracker markRegionAsCompacting(regionInfo); - long returnValue = runUpdateStats(tableName); - assertTrue("Update stats should have been skipped", returnValue >= UPDATE_STATS_SKIPPED); + Assert.assertEquals("Row count didn't match", COMPACTION_UPDATE_STATS_ROW_COUNT, runUpdateStats(tableName)); StatisticsCollectionRunTracker tracker = StatisticsCollectionRunTracker.getInstance(new Configuration()); // assert that the tracker state was cleared. @@ -113,10 +114,10 @@ public class StatisticsCollectionRunTrackerIT extends BaseOwnClusterHBaseManaged String tableName = "testUpdateStatsPreventsAnotherUpdateStatsFromRunning".toUpperCase(); HRegionInfo regionInfo = createTableAndGetRegion(tableName); markRunningUpdateStats(regionInfo); + Assert.assertEquals("Row count didn't match", CONCURRENT_UPDATE_STATS_ROW_COUNT, + runUpdateStats(tableName)); StatisticsCollectionRunTracker tracker = StatisticsCollectionRunTracker.getInstance(new Configuration()); - assertTrue("Update stats should have been skipped", runUpdateStats(tableName) >= UPDATE_STATS_SKIPPED); - // assert that running the concurrent and race-losing update stats didn't clear the region // from the tracker. If the method returned true it means the tracker was still tracking // the region. Slightly counter-intuitive, yes. http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java index fe95470..aba9c11 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/IndexToolIT.java @@ -108,8 +108,7 @@ public class IndexToolIT extends BaseOwnClusterHBaseManagedTimeIT { final String fullTableName = SchemaUtil.getTableName(schemaName, dataTable); final String indxTable = String.format("%s_%s", dataTable, "INDX"); Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); - props.setProperty(QueryServices.TRANSACTIONS_ENABLED, Boolean.TRUE.toString()); - props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.FALSE.toString()); + props.setProperty(QueryServices.TRANSACTIONS_ENABLED, "true"); Connection conn = DriverManager.getConnection(getUrl(), props); Statement stmt = conn.createStatement(); try { http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java index 8125007..0791479 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableIndexToolIT.java @@ -61,7 +61,6 @@ public class MutableIndexToolIT extends BaseOwnClusterHBaseManagedTimeIT { final String dataTable = "DATA_TABLE5"; final String indxTable = String.format("%s_%s",dataTable,"INDX"); Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); - props.setProperty(QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, Boolean.FALSE.toString()); Connection conn = DriverManager.getConnection(getUrl(), props); Statement stmt = conn.createStatement(); try { http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectionDisabledIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectionDisabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectionDisabledIT.java index c3cdbc0..a92a665 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectionDisabledIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectionDisabledIT.java @@ -17,27 +17,24 @@ */ package org.apache.phoenix.end2end; -import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - import java.sql.Connection; import java.sql.DriverManager; +import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import java.util.Map; import java.util.Properties; -import org.apache.phoenix.jdbc.PhoenixConnection; +import com.google.common.collect.Maps; import org.apache.phoenix.query.QueryServices; import org.apache.phoenix.util.PropertiesUtil; import org.apache.phoenix.util.ReadOnlyProps; import org.junit.BeforeClass; import org.junit.Test; -import com.google.common.collect.Maps; +import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES; +import static org.junit.Assert.assertFalse; /** * Verifies that statistics are not collected if they are disabled via a setting @@ -57,21 +54,15 @@ public class StatsCollectionDisabledIT extends StatsCollectorAbstractIT { public void testStatisticsAreNotWritten() throws SQLException { Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); Connection conn = DriverManager.getConnection(getUrl(), props); - assertFalse(conn.unwrap(PhoenixConnection.class).getQueryServices().areStatsEnabled()); Statement stmt = conn.createStatement(); - stmt.execute("CREATE TABLE T1 (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR) SALT_BUCKETS=3"); + stmt.execute("CREATE TABLE T1 (ID INTEGER NOT NULL PRIMARY KEY, NAME VARCHAR)"); stmt.execute("UPSERT INTO T1 VALUES (1, 'NAME1')"); stmt.execute("UPSERT INTO T1 VALUES (2, 'NAME2')"); stmt.execute("UPSERT INTO T1 VALUES (3, 'NAME3')"); conn.commit(); stmt.execute("UPDATE STATISTICS T1"); - assertFalse(conn.unwrap(PhoenixConnection.class).getQueryServices().areStatsEnabled()); ResultSet rs = stmt.executeQuery("SELECT * FROM SYSTEM.STATS"); assertFalse(rs.next()); - rs = conn.createStatement().executeQuery("SELECT count(*) FROM T1"); - assertTrue(rs.next()); - assertEquals(3,rs.getInt(1)); - assertFalse(conn.unwrap(PhoenixConnection.class).getQueryServices().areStatsEnabled()); rs.close(); stmt.close(); conn.close(); http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java index bfb8af1..ff2bcda 100644 --- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java +++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java @@ -95,7 +95,6 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT { Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES); // props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10)); conn = DriverManager.getConnection(getUrl(), props); - assertTrue(conn.unwrap(PhoenixConnection.class).getQueryServices().areStatsEnabled()); conn.createStatement().execute( "CREATE TABLE " + fullTableName +" ( k VARCHAR, a_string_array VARCHAR(100) ARRAY[4], b_string_array VARCHAR(100) ARRAY[4] \n" + " CONSTRAINT pk PRIMARY KEY (k, b_string_array DESC))" @@ -106,7 +105,6 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT { // CAll the update statistics query here. If already major compaction has run this will not get executed. stmt = conn.prepareStatement("UPDATE STATISTICS " + tableName); stmt.execute(); - assertTrue(conn.unwrap(PhoenixConnection.class).getQueryServices().areStatsEnabled()); stmt = upsertStmt(conn, tableName); stmt.setString(1, "z"); s = new String[] { "xyz", "def", "ghi", "jkll", null, null, "xxx" }; @@ -123,7 +121,6 @@ public class StatsCollectorIT extends StatsCollectorAbstractIT { stmt.execute(); rs = conn.createStatement().executeQuery("SELECT k FROM " + tableName); assertTrue(rs.next()); - assertTrue(conn.unwrap(PhoenixConnection.class).getQueryServices().areStatsEnabled()); conn.close(); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java index 6965216..9822807 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java @@ -140,6 +140,7 @@ import org.apache.phoenix.expression.LiteralExpression; import org.apache.phoenix.hbase.index.covered.update.ColumnReference; import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder; import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; +import org.apache.phoenix.hbase.index.util.IndexManagementUtil; import org.apache.phoenix.index.IndexMaintainer; import org.apache.phoenix.jdbc.PhoenixConnection; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; @@ -2718,7 +2719,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso RpcCallback done) { GetVersionResponse.Builder builder = GetVersionResponse.newBuilder(); - long version = MetaDataUtil.encodeVersion(env.getHBaseVersion(), env.getConfiguration()); + // The first 3 bytes of the long is used to encoding the HBase version as major.minor.patch. + // The next 4 bytes of the value is used to encode the Phoenix version as major.minor.patch. + long version = MetaDataUtil.encodeHBaseAndPhoenixVersions(this.env.getHBaseVersion()); + + // The last byte is used to communicate whether or not mutable secondary indexing + // was configured properly. + version = + MetaDataUtil.encodeHasIndexWALCodec(version, + IndexManagementUtil.isWALEditCodecSet(this.env.getConfiguration())); + builder.setVersion(version); done.run(builder.build()); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java index a800f16..5481910 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java @@ -22,9 +22,8 @@ import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN; import static org.apache.phoenix.query.QueryConstants.SINGLE_COLUMN_FAMILY; import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY; import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB; -import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.UPDATE_STATS_DISABLED; -import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.UPDATE_STATS_RUN; -import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.UPDATE_STATS_SKIPPED; +import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT; +import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -60,6 +59,7 @@ import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.ScanType; import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.io.WritableUtils; import org.apache.phoenix.coprocessor.generated.PTableProtos; import org.apache.phoenix.exception.DataExceedsCapacityException; @@ -90,7 +90,6 @@ import org.apache.phoenix.schema.ValueSchema.Field; import org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker; import org.apache.phoenix.schema.stats.StatisticsCollector; import org.apache.phoenix.schema.stats.StatisticsCollectorFactory; -import org.apache.phoenix.schema.stats.StatisticsUtil; import org.apache.phoenix.schema.tuple.MultiKeyValueTuple; import org.apache.phoenix.schema.types.PBinary; import org.apache.phoenix.schema.types.PChar; @@ -610,6 +609,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver InternalScanner internalScanner = scanner; if (scanType.equals(ScanType.COMPACT_DROP_DELETES)) { try { + Pair mergeRegions = null; long clientTimeStamp = TimeKeeper.SYSTEM.getCurrentTime(); StatisticsCollector stats = StatisticsCollectorFactory.createStatisticsCollector( c.getEnvironment(), table.getNameAsString(), clientTimeStamp, @@ -645,23 +645,22 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver if (asyncBytes != null) { async = Bytes.toBoolean(asyncBytes); } - long returnValue = UPDATE_STATS_RUN; // in case of async, we report 1 as number of rows updated - boolean statsEnabled = StatisticsUtil.isStatsEnabled(config); + long rowCount = 0; // in case of async, we report 0 as number of rows updated StatisticsCollectionRunTracker statsRunTracker = StatisticsCollectionRunTracker.getInstance(config); - boolean runUpdateStats = statsEnabled && statsRunTracker.addUpdateStatsCommandRegion(region.getRegionInfo()); + boolean runUpdateStats = statsRunTracker.addUpdateStatsCommandRegion(region.getRegionInfo()); if (runUpdateStats) { if (!async) { - returnValue = callable.call(); + rowCount = callable.call(); } else { statsRunTracker.runTask(callable); } } else { - returnValue = statsEnabled ? UPDATE_STATS_SKIPPED : UPDATE_STATS_DISABLED; - logger.info("UPDATE STATISTICS didn't run because " + (statsEnabled ? " another UPDATE STATISTICS command was already running on the region " : " stats are disabled ") + rowCount = CONCURRENT_UPDATE_STATS_ROW_COUNT; + logger.info("UPDATE STATISTICS didn't run because another UPDATE STATISTICS command was already running on the region " + region.getRegionInfo().getRegionNameAsString()); } - byte[] rowCountBytes = PLong.INSTANCE.toBytes(Long.valueOf(returnValue)); + byte[] rowCountBytes = PLong.INSTANCE.toBytes(Long.valueOf(rowCount)); final KeyValue aggKeyValue = KeyValueUtil.newKeyValue(UNGROUPED_AGG_ROW_KEY, SINGLE_COLUMN_FAMILY, SINGLE_COLUMN, AGG_TIMESTAMP, rowCountBytes, 0, rowCountBytes.length); @@ -731,22 +730,22 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver region.startRegionOperation(); boolean hasMore = false; boolean noErrors = false; - long rowCount = 0; boolean compactionRunning = areStatsBeingCollectedViaCompaction(); + long rowCount = 0; try { if (!compactionRunning) { synchronized (innerScanner) { do { List results = new ArrayList(); hasMore = innerScanner.nextRaw(results); - rowCount++; stats.collectStatistics(results); + rowCount++; compactionRunning = areStatsBeingCollectedViaCompaction(); } while (hasMore && !compactionRunning); noErrors = true; } } - return compactionRunning ? UPDATE_STATS_SKIPPED : UPDATE_STATS_RUN; + return compactionRunning ? COMPACTION_UPDATE_STATS_ROW_COUNT : rowCount; } catch (IOException e) { logger.error("IOException in update stats: " + Throwables.getStackTraceAsString(e)); throw e; http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java index 8d1a365..fc3edbe 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java @@ -122,7 +122,6 @@ public abstract class BaseResultIterators extends ExplainTable implements Result private final List>>>> allFutures; private long estimatedRows; private long estimatedSize; - private boolean areStatsEnabled; static final Function TO_KEY_RANGE = new Function() { @Override @@ -370,11 +369,6 @@ public abstract class BaseResultIterators extends ExplainTable implements Result return scans; } - // TODO: add to ResultIterators and QueryPlan interfaces? - public boolean areStatsEnabled() { - return this.areStatsEnabled; - } - private static List toBoundaries(List regionLocations) { int nBoundaries = regionLocations.size() - 1; List ranges = Lists.newArrayListWithExpectedSize(nBoundaries); @@ -530,17 +524,6 @@ public abstract class BaseResultIterators extends ExplainTable implements Result ImmutableBytesWritable currentGuidePost = ByteUtil.EMPTY_IMMUTABLE_BYTE_ARRAY; List scans = Lists.newArrayListWithExpectedSize(estGuidepostsPerRegion); ImmutableBytesWritable guidePosts = gps.getGuidePosts(); - // If we have any guideposts, then we can definitely say that stats are enabled. - // If we have no guideposts, though, we cannot assume that stats are disabled, - // as the table may just be too small to have them. - if (guidePosts.getLength() > 0) { - areStatsEnabled = true; - // It's possible that the server was bounced and stats have changed - // to become enabled without a client bounce. - this.context.getConnection().getQueryServices().setStatsEnabled(true); - } else { - areStatsEnabled = this.context.getConnection().getQueryServices().areStatsEnabled(); - } ByteArrayInputStream stream = null; DataInput input = null; PrefixByteDecoder decoder = null; @@ -868,18 +851,14 @@ public abstract class BaseResultIterators extends ExplainTable implements Result @Override public void explain(List planSteps) { - ConnectionQueryServices services = context.getConnection().getQueryServices(); - boolean displayChunkCount = services.getProps().getBoolean( + boolean displayChunkCount = context.getConnection().getQueryServices().getProps().getBoolean( QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB, QueryServicesOptions.DEFAULT_EXPLAIN_CHUNK_COUNT); - boolean displayRowCount = services.getProps().getBoolean( - QueryServices.EXPLAIN_ROW_COUNT_ATTRIB, - QueryServicesOptions.DEFAULT_EXPLAIN_ROW_COUNT); StringBuilder buf = new StringBuilder(); buf.append("CLIENT "); if (displayChunkCount) { buf.append(this.splits.size()).append("-CHUNK "); - if (displayRowCount && areStatsEnabled) { + if (estimatedRows > 0) { buf.append(estimatedRows).append(" ROWS "); buf.append(estimatedSize).append(" BYTES "); } http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java index 8a05f03..b5f1f85 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java @@ -123,7 +123,4 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated TransactionSystemClient getTransactionSystemClient(); public long getRenewLeaseThresholdMilliSeconds(); public boolean isRenewingLeasesEnabled(); - - public boolean areStatsEnabled(); - public void setStatsEnabled(boolean statsEnabled); } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java index 26e0783..4c48179 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java @@ -241,7 +241,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement private TransactionServiceClient txServiceClient; private volatile boolean initialized; private volatile int nSequenceSaltBuckets; - private volatile boolean areStatsEnabled; // writes guarded by "this" private volatile boolean closed; @@ -1110,7 +1109,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement private void checkClientServerCompatibility() throws SQLException { StringBuilder buf = new StringBuilder("The following servers require an updated " + QueryConstants.DEFAULT_COPROCESS_PATH + " to be put in the classpath of HBase: "); boolean isIncompatible = false; - boolean areStatsEnabled = false; int minHBaseVersion = Integer.MAX_VALUE; try { List locations = this.getAllTableRegions(SYSTEM_CATALOG_NAME_BYTES); @@ -1144,23 +1142,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement }); for (Map.Entry result : results.entrySet()) { // This is the "phoenix.jar" is in-place, but server is out-of-sync with client case. - long version = result.getValue(); - // Set stats as being enabled if enabled on *any* region server (though it should - // really match across all regions servers). - areStatsEnabled |= MetaDataUtil.decodeStatsEnabled(version); - if (!isCompatible(version)) { + if (!isCompatible(result.getValue())) { isIncompatible = true; HRegionLocation name = regionMap.get(result.getKey()); buf.append(name); buf.append(';'); } - hasIndexWALCodec &= hasIndexWALCodec(version); - if (minHBaseVersion > MetaDataUtil.decodeHBaseVersion(version)) { - minHBaseVersion = MetaDataUtil.decodeHBaseVersion(version); + hasIndexWALCodec &= hasIndexWALCodec(result.getValue()); + if (minHBaseVersion > MetaDataUtil.decodeHBaseVersion(result.getValue())) { + minHBaseVersion = MetaDataUtil.decodeHBaseVersion(result.getValue()); } } lowestClusterHBaseVersion = minHBaseVersion; - this.areStatsEnabled = areStatsEnabled; } catch (SQLException e) { throw e; } catch (Throwable t) { @@ -3383,14 +3376,4 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement } } - @Override - public boolean areStatsEnabled() { - return areStatsEnabled; - } - - @Override - public void setStatsEnabled(boolean statsEnabled) { - this.areStatsEnabled = statsEnabled; - } - } http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java index 453d04f..b4bbe1f 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java @@ -76,7 +76,6 @@ import org.apache.phoenix.schema.SequenceNotFoundException; import org.apache.phoenix.schema.TableAlreadyExistsException; import org.apache.phoenix.schema.TableNotFoundException; import org.apache.phoenix.schema.stats.PTableStats; -import org.apache.phoenix.schema.stats.StatisticsUtil; import org.apache.phoenix.util.JDBCUtil; import org.apache.phoenix.util.MetaDataUtil; import org.apache.phoenix.util.PhoenixRuntime; @@ -107,8 +106,7 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple private final Map sequenceMap = Maps.newHashMap(); private final String userName; private final TransactionSystemClient txSystemClient; - private final KeyValueBuilder kvBuilder; - private volatile boolean areStatsEnabled; + private KeyValueBuilder kvBuilder; private volatile boolean initialized; private volatile SQLException initializationException; private final Map> tableSplits = Maps.newHashMap(); @@ -138,8 +136,6 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration(config); TransactionManager txnManager = new TransactionManager(config); this.txSystemClient = new InMemoryTxSystemClient(txnManager); - // Just check the properties on the client side (instead of normally the server side) - this.areStatsEnabled = StatisticsUtil.isStatsEnabled(config); } private PMetaData newEmptyMetaData() { @@ -531,7 +527,6 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple return txSystemClient; } - @Override public MetaDataMutationResult createFunction(List functionData, PFunction function, boolean temporary) throws SQLException { return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, 0l, null); @@ -583,7 +578,6 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple return false; } - @Override public HRegionLocation getTableRegionLocation(byte[] tableName, byte[] row) throws SQLException { List regions = tableSplits.get(Bytes.toString(tableName)); if (regions != null) { @@ -598,14 +592,4 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple new HRegionInfo(TableName.valueOf(tableName), HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW), SERVER_NAME, -1); } - - @Override - public boolean areStatsEnabled() { - return areStatsEnabled; - } - - @Override - public void setStatsEnabled(boolean statsEnabled) { - this.areStatsEnabled = statsEnabled; - } } http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java index 2d0f677..4c7446b 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java @@ -310,14 +310,4 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple throws SQLException { return getDelegate().getTableRegionLocation(tableName, row); } - - @Override - public boolean areStatsEnabled() { - return getDelegate().areStatsEnabled(); - } - - @Override - public void setStatsEnabled(boolean statsEnabled) { - getDelegate().setStatsEnabled(statsEnabled); - } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java index 0701c10..000c63c 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java @@ -162,7 +162,6 @@ public interface QueryServices extends SQLCloseable { public static final String SEQUENCE_SALT_BUCKETS_ATTRIB = "phoenix.sequence.saltBuckets"; public static final String COPROCESSOR_PRIORITY_ATTRIB = "phoenix.coprocessor.priority"; public static final String EXPLAIN_CHUNK_COUNT_ATTRIB = "phoenix.explain.displayChunkCount"; - public static final String EXPLAIN_ROW_COUNT_ATTRIB = "phoenix.explain.displayRowCount"; public static final String ALLOW_ONLINE_TABLE_SCHEMA_UPDATE = "hbase.online.schema.update.enable"; public static final String NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK = "phoenix.schema.change.retries"; public static final String DELAY_FOR_SCHEMA_UPDATE_CHECK = "phoenix.schema.change.delay"; http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java index 726fe57..ea07f25 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java @@ -29,7 +29,6 @@ import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB import static org.apache.phoenix.query.QueryServices.DELAY_FOR_SCHEMA_UPDATE_CHECK; import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB; import static org.apache.phoenix.query.QueryServices.EXPLAIN_CHUNK_COUNT_ATTRIB; -import static org.apache.phoenix.query.QueryServices.EXPLAIN_ROW_COUNT_ATTRIB; import static org.apache.phoenix.query.QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB; import static org.apache.phoenix.query.QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB; import static org.apache.phoenix.query.QueryServices.GLOBAL_METRICS_ENABLED; @@ -198,7 +197,6 @@ public class QueryServicesOptions { */ public static final int DEFAULT_COPROCESSOR_PRIORITY = Coprocessor.PRIORITY_SYSTEM/2 + Coprocessor.PRIORITY_USER/2; // Divide individually to prevent any overflow public static final boolean DEFAULT_EXPLAIN_CHUNK_COUNT = true; - public static final boolean DEFAULT_EXPLAIN_ROW_COUNT = true; public static final boolean DEFAULT_ALLOW_ONLINE_TABLE_SCHEMA_UPDATE = true; public static final int DEFAULT_RETRIES_FOR_SCHEMA_UPDATE_CHECK = 10; public static final long DEFAULT_DELAY_FOR_SCHEMA_UPDATE_CHECK = 5 * 1000; // 5 seconds. @@ -576,11 +574,6 @@ public class QueryServicesOptions { return this; } - public QueryServicesOptions setExplainRowCount(boolean showRowCount) { - config.setBoolean(EXPLAIN_ROW_COUNT_ATTRIB, showRowCount); - return this; - } - public QueryServicesOptions setAllowOnlineSchemaUpdate(boolean allow) { config.setBoolean(ALLOW_ONLINE_TABLE_SCHEMA_UPDATE, allow); return this; http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java index eee42ea..7f3f850 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java @@ -182,7 +182,6 @@ import org.apache.phoenix.schema.PTable.IndexType; import org.apache.phoenix.schema.PTable.LinkType; import org.apache.phoenix.schema.PTable.ViewType; import org.apache.phoenix.schema.stats.PTableStats; -import org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker; import org.apache.phoenix.schema.types.PDataType; import org.apache.phoenix.schema.types.PDate; import org.apache.phoenix.schema.types.PInteger; @@ -956,7 +955,6 @@ public class MetaDataClient { } MutationState mutationState = plan.execute(); rowCount = mutationState.getUpdateCount(); - this.getConnection().getQueryServices().setStatsEnabled(rowCount != StatisticsCollectionRunTracker.UPDATE_STATS_DISABLED); } /* http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectionRunTracker.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectionRunTracker.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectionRunTracker.java index 6413e37..4ed3325 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectionRunTracker.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsCollectionRunTracker.java @@ -24,6 +24,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ThreadFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HRegionInfo; @@ -44,9 +45,9 @@ public class StatisticsCollectionRunTracker { .newSetFromMap(new ConcurrentHashMap()); private final ExecutorService executor; - public static final long UPDATE_STATS_RUN = 1L; - public static final long UPDATE_STATS_SKIPPED = 100000L; - public static final long UPDATE_STATS_DISABLED = 0; + // Constants added for testing purposes + public static final long CONCURRENT_UPDATE_STATS_ROW_COUNT = -100l; + public static final long COMPACTION_UPDATE_STATS_ROW_COUNT = -200l; public static StatisticsCollectionRunTracker getInstance(Configuration config) { StatisticsCollectionRunTracker result = INSTANCE; http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java index 9dd72e8..5b47104 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsUtil.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.SortedMap; import java.util.TreeMap; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellScanner; import org.apache.hadoop.hbase.HConstants; @@ -41,8 +40,6 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol; import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; import org.apache.phoenix.query.QueryConstants; -import org.apache.phoenix.query.QueryServices; -import org.apache.phoenix.query.QueryServicesOptions; import org.apache.phoenix.schema.SortOrder; import org.apache.phoenix.schema.types.PLong; import org.apache.phoenix.util.ByteUtil; @@ -223,17 +220,4 @@ public class StatisticsUtil { ptr.set(row, gpOffset, row.length - gpOffset); return ByteUtil.copyKeyBytesIfNecessary(ptr); } - - public static boolean isStatsEnabled(Configuration conf) { - if (conf.getBoolean(QueryServices.STATS_ENABLED_ATTRIB, true)) { - if (conf.getInt(QueryServices.STATS_GUIDEPOST_PER_REGION_ATTRIB, - QueryServicesOptions.DEFAULT_STATS_GUIDEPOST_PER_REGION) != 1) { - if (conf.getLong(QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB, - QueryServicesOptions.DEFAULT_STATS_GUIDEPOST_WIDTH_BYTES) < HConstants.DEFAULT_MAX_FILE_SIZE) { - return true; - } - } - } - return false; - } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java index 5ee3c0a..6d5903d 100644 --- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java +++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.ipc.RemoteException; import org.apache.phoenix.coprocessor.MetaDataProtocol; import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; -import org.apache.phoenix.hbase.index.util.IndexManagementUtil; import org.apache.phoenix.hbase.index.util.KeyValueBuilder; import org.apache.phoenix.hbase.index.util.VersionUtil; import org.apache.phoenix.jdbc.PhoenixConnection; @@ -57,7 +56,6 @@ import org.apache.phoenix.schema.PTableType; import org.apache.phoenix.schema.SequenceKey; import org.apache.phoenix.schema.SortOrder; import org.apache.phoenix.schema.TableNotFoundException; -import org.apache.phoenix.schema.stats.StatisticsUtil; import org.apache.phoenix.schema.types.PBoolean; import org.apache.phoenix.schema.types.PDataType; import org.apache.phoenix.schema.types.PLong; @@ -105,25 +103,26 @@ public class MetaDataUtil { // The second byte in int would be the major version, 3rd byte minor version, and 4th byte // patch version. public static int decodePhoenixVersion(long version) { - return (int) ((version << Byte.SIZE * 4) >>> Byte.SIZE * 5); + return (int) ((version << Byte.SIZE * 3) >>> Byte.SIZE * 4); + } + + // TODO: generalize this to use two bytes to return a SQL error code instead + public static long encodeHasIndexWALCodec(long version, boolean isValid) { + if (!isValid) { + return version | 1; + } + return version; } public static boolean decodeHasIndexWALCodec(long version) { return (version & 0xF) == 0; } - // Given the encoded integer representing the phoenix version in the encoded version value. - // The second byte in int would be the major version, 3rd byte minor version, and 4th byte - // patch version. - public static boolean decodeStatsEnabled(long version) { - return ((int) ((version << Byte.SIZE * 3) >>> Byte.SIZE * 7) & 0x1) != 0; - } - // Given the encoded integer representing the client hbase version in the encoded version value. // The second byte in int would be the major version, 3rd byte minor version, and 4th byte // patch version. public static int decodeHBaseVersion(long version) { - return (int) (version >>> (Byte.SIZE * 5)); + return (int) (version >>> Byte.SIZE * 5); } public static String decodeHBaseVersionAsString(int version) { @@ -133,33 +132,17 @@ public class MetaDataUtil { return major + "." + minor + "." + patch; } - // The first 3 bytes of the long is used to encoding the HBase version as major.minor.patch. - // The next 4 bytes of the value is used to encode the Phoenix version as major.minor.patch. - /** - * Encode HBase and Phoenix version along with some server-side config information such - * as whether WAL codec is installed (necessary for non transactional, mutable secondar - * indexing), and whether stats are enabled. - * @param env RegionCoprocessorEnvironment to access HBase version and Configuration. - * @return long value sent back during initialization of a cluster connection. - */ - public static long encodeVersion(String hbaseVersionStr, Configuration config) { - long hbaseVersion = VersionUtil.encodeVersion(hbaseVersionStr); - long statsEnabled = StatisticsUtil.isStatsEnabled(config) ? 1 : 0; - long phoenixVersion = VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION, + public static int encodePhoenixVersion() { + return VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION, MetaDataProtocol.PHOENIX_PATCH_NUMBER); - long walCodec = IndexManagementUtil.isWALEditCodecSet(config) ? 0 : 1; - long version = - // Encode HBase major, minor, patch version - (hbaseVersion << (Byte.SIZE * 5)) - // Encode if stats are enabled on the server side - | (statsEnabled << (Byte.SIZE * 4)) - // Encode Phoenix major, minor, patch version - | (phoenixVersion << (Byte.SIZE * 1)) - // Encode whether or not non transactional, mutable secondary indexing was configured properly. - | walCodec; - return version; } - + + public static long encodeHBaseAndPhoenixVersions(String hbaseVersion) { + return (((long) VersionUtil.encodeVersion(hbaseVersion)) << (Byte.SIZE * 5)) | + (((long) VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION, + MetaDataProtocol.PHOENIX_PATCH_NUMBER)) << (Byte.SIZE * 1)); + } + public static void getTenantIdAndSchemaAndTableName(List tableMetadata, byte[][] rowKeyMetaData) { Mutation m = getTableHeaderRow(tableMetadata); getVarChars(m.getRow(), 3, rowKeyMetaData); http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java index 6ae655c..29a7001 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/query/QueryServicesTestImpl.java @@ -54,7 +54,6 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl { public static final long DEFAULT_MAX_CLIENT_METADATA_CACHE_SIZE = 1024L*1024L*2L; // 2 Mb public static final int DEFAULT_MIN_STATS_UPDATE_FREQ_MS = 0; public static final boolean DEFAULT_EXPLAIN_CHUNK_COUNT = false; // TODO: update explain plans in test and set to true - public static final boolean DEFAULT_EXPLAIN_ROW_COUNT = false; // TODO: update explain plans in test and set to true public static final String DEFAULT_EXTRA_JDBC_ARGUMENTS = PhoenixRuntime.PHOENIX_TEST_DRIVER_URL_PARAM; private static final boolean DEFAULT_RUN_UPDATE_STATS_ASYNC = false; private static final boolean DEFAULT_COMMIT_STATS_ASYNC = false; @@ -81,7 +80,6 @@ public final class QueryServicesTestImpl extends BaseQueryServicesImpl { private static QueryServicesOptions getDefaultServicesOptions() { return withDefaults() .setExplainChunkCount(DEFAULT_EXPLAIN_CHUNK_COUNT) - .setExplainRowCount(DEFAULT_EXPLAIN_ROW_COUNT) .setSequenceSaltBuckets(DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS) .setMinStatsUpdateFrequencyMs(DEFAULT_MIN_STATS_UPDATE_FREQ_MS) .setThreadPoolSize(DEFAULT_THREAD_POOL_SIZE) http://git-wip-us.apache.org/repos/asf/phoenix/blob/e00550c7/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java ---------------------------------------------------------------------- diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java index 7ffc054..1e06379 100644 --- a/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java +++ b/phoenix-core/src/test/java/org/apache/phoenix/util/MetaDataUtilTest.java @@ -21,19 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.VersionInfo; -import org.apache.phoenix.coprocessor.MetaDataProtocol; import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder; import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr; import org.apache.phoenix.hbase.index.util.KeyValueBuilder; import org.apache.phoenix.hbase.index.util.VersionUtil; import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData; -import org.apache.phoenix.query.HBaseFactoryProvider; -import org.apache.phoenix.query.QueryServices; import org.junit.Test; @@ -62,35 +58,6 @@ public class MetaDataUtilTest { assertFalse(MetaDataUtil.areClientAndServerCompatible(VersionUtil.encodeVersion(3,1,10), 3, 5)); } - @Test - public void testEncodeDecode() { - String hbaseVersionStr = "0.98.14"; - Configuration config = HBaseFactoryProvider.getConfigurationFactory().getConfiguration(); - config.setBoolean(QueryServices.STATS_ENABLED_ATTRIB, false); - - long version = MetaDataUtil.encodeVersion(hbaseVersionStr, config); - int hbaseVersion = MetaDataUtil.decodeHBaseVersion(version); - int expectedHBaseVersion = VersionUtil.encodeVersion(0, 98, 14); - assertEquals(expectedHBaseVersion, hbaseVersion); - boolean areStatsEnabled = MetaDataUtil.decodeStatsEnabled(version); - assertFalse(areStatsEnabled); - int phoenixVersion = MetaDataUtil.decodePhoenixVersion(version); - int expectedPhoenixVersion = VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION, - MetaDataProtocol.PHOENIX_PATCH_NUMBER); - assertEquals(expectedPhoenixVersion, phoenixVersion); - - config.setBoolean(QueryServices.STATS_ENABLED_ATTRIB, true); - version = MetaDataUtil.encodeVersion(hbaseVersionStr, config); - hbaseVersion = MetaDataUtil.decodeHBaseVersion(version); - expectedHBaseVersion = VersionUtil.encodeVersion(0, 98, 14); - assertEquals(expectedHBaseVersion, hbaseVersion); - areStatsEnabled = MetaDataUtil.decodeStatsEnabled(version); - assertTrue(areStatsEnabled); - phoenixVersion = MetaDataUtil.decodePhoenixVersion(version); - expectedPhoenixVersion = VersionUtil.encodeVersion(MetaDataProtocol.PHOENIX_MAJOR_VERSION, MetaDataProtocol.PHOENIX_MINOR_VERSION, - MetaDataProtocol.PHOENIX_PATCH_NUMBER); - assertEquals(expectedPhoenixVersion, phoenixVersion); - } /** * Ensure it supports {@link GenericKeyValueBuilder} * @throws Exception on failure