You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hawq.apache.org by pa...@apache.org on 2017/05/11 05:13:47 UTC

incubator-hawq git commit: HAWQ-1455. Wrong results on CTAS query over catalog

Repository: incubator-hawq
Updated Branches:
  refs/heads/master 94a2c65b7 -> 3461e6480


HAWQ-1455. Wrong results on CTAS query over catalog

This reverts the previous fix for HAWQ-512, however to HAWQ-512, it looks like
we could modify the lock related code following gpdb to do a real fix. Those code
was probably deleted during early hawq development.


Project: http://git-wip-us.apache.org/repos/asf/incubator-hawq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hawq/commit/3461e648
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hawq/tree/3461e648
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hawq/diff/3461e648

Branch: refs/heads/master
Commit: 3461e64801eb2299d46c86f47734b7f000152a10
Parents: 94a2c65
Author: Paul Guo <pa...@gmail.com>
Authored: Fri May 5 17:47:41 2017 +0800
Committer: Paul Guo <pa...@gmail.com>
Committed: Thu May 11 13:11:14 2017 +0800

----------------------------------------------------------------------
 src/backend/storage/lmgr/lock.c           | 45 +++++++++++++++++++++++---
 src/backend/tcop/postgres.c               |  2 +-
 src/backend/utils/cache/relcache.c        |  3 +-
 src/test/feature/catalog/ans/entrydb.ans  | 35 ++++++++++++++++++++
 src/test/feature/catalog/sql/entrydb.sql  | 22 +++++++++++++
 src/test/feature/catalog/test_entrydb.cpp | 37 +++++++++++++++++++++
 src/test/feature/full_tests.txt           |  2 +-
 7 files changed, 138 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/backend/storage/lmgr/lock.c
----------------------------------------------------------------------
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 7b43a10..85a6cd0 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -1019,7 +1019,6 @@ LockCheckConflicts(LockMethod lockMethodTable,
 				   PGPROC *proc)
 {
 	int			numLockModes = lockMethodTable->numLockModes;
-	LOCKMASK	myLocks;
 	LOCKMASK	otherLocks;
 	int			i;
 
@@ -1043,13 +1042,51 @@ LockCheckConflicts(LockMethod lockMethodTable,
 	 * to construct a conflict mask that does not reflect our own locks, but
 	 * only lock types held by other processes.
 	 */
-	myLocks = proclock->holdMask;
 	otherLocks = 0;
 	for (i = 1; i <= numLockModes; i++)
 	{
-		int			myHolding = (myLocks & LOCKBIT_ON(i)) ? 1 : 0;
+		int				ourHolding = 0;
 
-		if (lock->granted[i] > myHolding)
+		/*
+		 * If I'm not part of MPP session, consider I am only one process
+		 * in a session.
+		 */
+		if (proc->mppSessionId <= 0)
+		{
+			LOCKMASK	myLocks = proclock->holdMask;
+			if (myLocks & LOCKBIT_ON(i))
+				ourHolding = 1;
+		}
+		else
+		{
+			SHM_QUEUE	   *procLocks = &(lock->procLocks);
+			PROCLOCK	   *otherProclock =
+					(PROCLOCK *) SHMQueueNext(procLocks, procLocks,
+											  offsetof(PROCLOCK, lockLink));
+			/*
+			 * Go through the proclock queue in the lock.  otherProclock
+			 * may be this process itself.
+			 */
+			while (otherProclock)
+			{
+				PGPROC	   *otherProc = otherProclock->tag.myProc;
+
+				/*
+				 * If processes in my session are holding the lock, mask
+				 * it out so that we won't be blocked by them.
+				 */
+				if (otherProc->mppSessionId == proc->mppSessionId &&
+					otherProclock->holdMask & LOCKBIT_ON(i))
+					ourHolding++;
+
+				otherProclock =
+					(PROCLOCK *) SHMQueueNext(procLocks,
+											  &otherProclock->lockLink,
+											  offsetof(PROCLOCK, lockLink));
+			}
+		}
+
+		if (lock->granted[i] > ourHolding)
 			otherLocks |= LOCKBIT_ON(i);
 	}
 

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/backend/tcop/postgres.c
----------------------------------------------------------------------
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 488b8e1..0411c6b 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -4471,7 +4471,7 @@ PostgresMain(int argc, char *argv[], const char *username)
 	 * NOTE: we init entrydb as QE
 	 */
 	if (MyProcPort == NULL ||
-	    (AmIMaster() && Gp_role != GP_ROLE_EXECUTE) || AmIStandby() ||
+	    AmIMaster() || AmIStandby() ||
 	    MyProcPort->bootstrap_user == NULL){
 		am_superuser = InitPostgres(dbname, InvalidOid, username, NULL);
 	}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/backend/utils/cache/relcache.c
----------------------------------------------------------------------
diff --git a/src/backend/utils/cache/relcache.c b/src/backend/utils/cache/relcache.c
index 93ac2b5..5aa80aa 100644
--- a/src/backend/utils/cache/relcache.c
+++ b/src/backend/utils/cache/relcache.c
@@ -1344,8 +1344,7 @@ RelationInitPhysicalAddr(Relation relation)
 	if (relation->rd_rel->relisshared)
 		relation->rd_node.dbNode = InvalidOid;
 	else if (relation->rd_id < FirstNormalObjectId ||
-			(AmActiveMaster() && Gp_role != GP_ROLE_EXECUTE) ||
-			AmStandbyMaster())
+			AmActiveMaster() || AmStandbyMaster())
 		relation->rd_node.dbNode = MyDatabaseId;
 	else
 		relation->rd_node.dbNode = MyProcPort->dboid;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/test/feature/catalog/ans/entrydb.ans
----------------------------------------------------------------------
diff --git a/src/test/feature/catalog/ans/entrydb.ans b/src/test/feature/catalog/ans/entrydb.ans
new file mode 100644
index 0000000..26a5a67
--- /dev/null
+++ b/src/test/feature/catalog/ans/entrydb.ans
@@ -0,0 +1,35 @@
+-- test case for HAWQ-1455 (Wrong results on CTAS query over catalog)
+create temp table entrydb_t1 (entrydb_tta varchar, entrydb_ttb varchar);
+CREATE TABLE
+create temp table entrydb_t2 (entrydb_tta varchar, entrydb_ttb varchar);
+CREATE TABLE
+insert into entrydb_t1 values('entrydb_a', '1');
+INSERT 0 1
+insert into entrydb_t1 values('entrydb_a', '2');
+INSERT 0 1
+insert into entrydb_t1 values('entrydb_tta', '3');
+INSERT 0 1
+insert into entrydb_t1 values('entrydb_ttb', '4');
+INSERT 0 1
+insert into entrydb_t2 select pg_attribute.attname, entrydb_t1.entrydb_ttb from pg_attribute join entrydb_t1 on pg_attribute.attname = entrydb_t1.entrydb_tta;
+INSERT 0 4
+-- test case for HAWQ-512 (Query hang due to deadlock in entrydb catalog access)
+create table entrydb_t3 (key int, value int) distributed randomly;
+CREATE TABLE
+insert into entrydb_t3 values (1, 0);
+INSERT 0 1
+begin;
+BEGIN
+alter table entrydb_t3 set distributed by (key);
+ALTER TABLE
+select entrydb_t4.key FROM entrydb_t3 AS entrydb_t4, (select generate_series(1, 2)::int as key, 0::int as value) as entrydb_t5, (select generate_series(1, 2)::int as key, 0::int as value) as entrydb_t6 where entrydb_t4.value = entrydb_t5.value and entrydb_t4.value = entrydb_t6.value;
+ key 
+-----
+   1
+   1
+   1
+   1
+(4 rows)
+
+commit;
+COMMIT

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/test/feature/catalog/sql/entrydb.sql
----------------------------------------------------------------------
diff --git a/src/test/feature/catalog/sql/entrydb.sql b/src/test/feature/catalog/sql/entrydb.sql
new file mode 100644
index 0000000..cf3cd2c
--- /dev/null
+++ b/src/test/feature/catalog/sql/entrydb.sql
@@ -0,0 +1,22 @@
+
+-- test case for HAWQ-1455 (Wrong results on CTAS query over catalog)
+create temp table entrydb_t1 (entrydb_tta varchar, entrydb_ttb varchar);
+create temp table entrydb_t2 (entrydb_tta varchar, entrydb_ttb varchar);
+insert into entrydb_t1 values('entrydb_a', '1');
+insert into entrydb_t1 values('entrydb_a', '2');
+insert into entrydb_t1 values('entrydb_tta', '3');
+insert into entrydb_t1 values('entrydb_ttb', '4');
+
+insert into entrydb_t2 select pg_attribute.attname, entrydb_t1.entrydb_ttb from pg_attribute join entrydb_t1 on pg_attribute.attname = entrydb_t1.entrydb_tta;
+
+-- test case for HAWQ-512 (Query hang due to deadlock in entrydb catalog access)
+create table entrydb_t3 (key int, value int) distributed randomly;
+insert into entrydb_t3 values (1, 0);
+
+begin;
+
+alter table entrydb_t3 set distributed by (key);
+
+select entrydb_t4.key FROM entrydb_t3 AS entrydb_t4, (select generate_series(1, 2)::int as key, 0::int as value) as entrydb_t5, (select generate_series(1, 2)::int as key, 0::int as value) as entrydb_t6 where entrydb_t4.value = entrydb_t5.value and entrydb_t4.value = entrydb_t6.value;
+
+commit;

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/test/feature/catalog/test_entrydb.cpp
----------------------------------------------------------------------
diff --git a/src/test/feature/catalog/test_entrydb.cpp b/src/test/feature/catalog/test_entrydb.cpp
new file mode 100644
index 0000000..01ba6d0
--- /dev/null
+++ b/src/test/feature/catalog/test_entrydb.cpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include "gtest/gtest.h"
+
+#include "lib/sql_util.h"
+
+using std::string;
+
+class TestEntrydb: public ::testing::Test
+{
+	public:
+		TestEntrydb() {};
+		~TestEntrydb() {};
+};
+
+TEST_F(TestEntrydb, entrydb)
+{
+	hawq::test::SQLUtility util;
+
+	util.execSQLFile("catalog/sql/entrydb.sql", "catalog/ans/entrydb.ans");
+}

http://git-wip-us.apache.org/repos/asf/incubator-hawq/blob/3461e648/src/test/feature/full_tests.txt
----------------------------------------------------------------------
diff --git a/src/test/feature/full_tests.txt b/src/test/feature/full_tests.txt
index beda4e8..19cd9d6 100644
--- a/src/test/feature/full_tests.txt
+++ b/src/test/feature/full_tests.txt
@@ -3,4 +3,4 @@
 #you can have several PARALLEL or SRRIAL
 
 PARALLEL=TestErrorTable.*:TestPreparedStatement.*:TestUDF.*:TestAOSnappy.*:TestAlterOwner.*:TestAlterTable.*:TestCreateTable.*:TestGuc.*:TestType.*:TestDatabase.*:TestParquet.*:TestPartition.*:TestSubplan.*:TestAggregate.*:TestCreateTypeComposite.*:TestGpDistRandom.*:TestInformationSchema.*:TestQueryInsert.*:TestQueryNestedCaseNull.*:TestQueryPolymorphism.*:TestQueryPortal.*:TestQueryPrepare.*:TestQuerySequence.*:TestCommonLib.*:TestToast.*:TestTransaction.*:TestCommand.*:TestCopy.*:TestParser.*:TestHawqRegister.*:TestRegex.*
-SERIAL=TestExternalOid.TestExternalOidAll:TestExternalTable.TestExternalTableAll:TestTemp.BasicTest:TestRowTypes.*
+SERIAL=TestExternalOid.TestExternalOidAll:TestExternalTable.TestExternalTableAll:TestTemp.BasicTest:TestRowTypes.*:TestEntrydb.entrydb