1. 19 4月, 2018 7 次提交
    • D
      Speed up dispatcher detection of segment state changes · 85101317
      David Kimura 提交于
      Dispatcher has DISPATCH_WAIT_TIMEOUT_MSEC (current value is 2000) as poll
      timeout. It waited for 30 iterations of poll to timeout before checking the
      segment status. And then initiated fts probe before checking the segment
      status. As a result it took ~minute for query to fail in case of segment
      failures.
      
      This commit updates to check segment status on every poll timeout. It also
      leverages fts version to optimize whether to check segments. It avoids
      performing fts probe, instead it relies on fts to be called on regular
      intervals and provide cached results.
      
      With this change test time for twophase_tolerance_with_mirror_promotion was cut
      down by ~2 minutes.
      Co-authored-by: NAshwin Agrawal <aagrawal@pivotal.io>
      85101317
    • D
      Add tests to cover mirror promotion scenario · 19e0f385
      David Kimura 提交于
      The goal of these tests is to validate that existing transactions on the
      primary are not hung when the mirror gets promoted. In order to validate
      this, mirror promotion is triggered at the following two-phase commit
      points:
        1) if the transaction hasn't prepared, then it should be aborted
        2) if the transaction is already prepared, then it should complete commit
           on the mirror
        3) if the transaction has committed on the primary, but not acknowledged
           to the master then it should complete the commit on the mirror
      Co-authored-by: NAshwin Agrawal <aagrawal@pivotal.io>
      19e0f385
    • A
      Pull start_prepare fault before HOLD_INTERRUPTS. · ec8aa4e9
      Ashwin Agrawal 提交于
      Sometimes test need to use this fault to suspend and then stop the database. If
      this fault is inside the HOLD_INTERRUPTS (which is where its before this
      change), CHECK_FOR_INTERRUPTS doesn't do anything. Hence stopping of segment
      takes super long time as kill -9 needs to be enforced by gpstop after timeout.
      Co-authored-by: NDavid Kimura <dkimura@pivotal.io>
      ec8aa4e9
    • N
      resgroup: fix an access to uninitialized address. · 56c20709
      Ning Yu 提交于
      In ResGroupDropFinish() uninitialized memory address can be accessed due
      to some reasons:
      
      1. the group pointer is not initialized on segments;
      2. the hash table node pointed by group is recycled in removeGroup();
      
      This invalid access can cause crash issue on segments.
      
      Also move some global vars to resgroup.c,
      They were put in resgroup-ops-linux.c, which was only compiled and
      linked on linux, so on other OS like macos the vars can not be found.
      56c20709
    • J
      User query can use global shared memory across resource group when... · 3be31490
      Jialun 提交于
      User query can use global shared memory across resource group when available(Same as the PR 4843, just made the test cases stable) (#4866)
      
      1) Global shared memory will be used if the query has run out of
      	the group shared memory.
      2) The limit of memory_spill_ratio changes to [0, INT_MAX], because
      	global shared memory can be allocated, 100% limitation is not
      	make sense.
      3) Using Atomic Operation & "Compare And Save" instead of lock to
      	get high performance.
      4) Modify the test cases according to the new rules.
      3be31490
    • N
      resgroup: fix a boot failure when cgroup is not mounted. · 4c8f28b0
      Ning Yu 提交于
      Cgroup mount point is detected during startup, but due to a bug the
      mount point was used even if cgroup is not mounted at all.
      
      Fixed by correcting the checking logic.
      4c8f28b0
    • N
      resgroup: backward compatibility for memory auditor · f2f86174
      Ning Yu 提交于
      Memory auditor was a new feature introduced to allow external components
      (e.g. pl/container) managed by resource group.  This feature requires a
      new gpdb dir to be created in cgroup memory controller, however on 5X
      branch unless the users created this new dir manually then the upgrade
      from a previous version would fail.
      
      In this commit we provide backward compatibility by checking the release
      version:
      
      - on 6X and master branches the memory auditor feature is always enabled
        so the new gpdb dir is mandatory;
      - on 5X branch only if the new gpdb dir is created with proper
        permissions the memory auditor feature could be enabled, when it's
        disabled `CREATE RESOURCE GROUP WITH (memory_auditor='cgroup') will fail
        with guide information on how to enable it;
      
      Binary swap tests are also provided to verify backward compatibility in
      future releases.  As cgroup need to be configured to enable resgroup we
      split the resgroup binary swap tests into two parts:
      
      - resqueue mode only tests which can be triggered in the
        icw_gporca_centos6 pipeline job after the ICW tests, these parts have
        no requirements on cgroup;
      - complete resqueue & resgroup modes tests which can be triggered in the
        mpp_resource_group_centos{6,7} pipeline jobs after the resgroup tests,
        these parts need cgroup to be properly configured;
      f2f86174
  2. 18 4月, 2018 1 次提交
    • T
      Build backend/parser/scan.l and interfaces/ecpg/preproc/pgc.l standalone. · fa3ec7b2
      Tom Lane 提交于
      We are developing on a machine with flex 2.6.4 and we couldn't build.
      This was fixed in 72b1e3a2 in upstream (9.6 at the time) in 2016, but
      wasn't backported to 9.1 or older.
      
      Original commit message:
      > Now that we know about the %top{} trick, we can revert to building flex
      > lexers as separate .o files.  This is worth doing for a couple of reasons
      > besides sheer cleanliness.  We can narrow the scope of the -Wno-error flag
      > that's forced on scan.c.  Also, since these grammar and lexer files are
      > so large, splitting them into separate build targets should have some
      > advantages in build speed, particularly in parallel or ccache'd builds.
      >
      > We have quite a few other .l files that could be changed likewise, but the
      > above arguments don't apply to them, so the benefit of fixing them seems
      > pretty minimal.  Leave the rest for some other day.
      
      (cherry picked from commit 72b1e3a2)
      
      Fixes greenplum-db/gpdb#4863
      Co-authored-by: NJesse Zhang <sbjesse@gmail.com>
      Co-authored-by: NAsim R P <apraveen@pivotal.io>
      fa3ec7b2
  3. 17 4月, 2018 3 次提交
    • J
      Revert "User query can use global shared memory across resource group when... · b0333c53
      Jialun 提交于
      Revert "User query can use global shared memory across resource group when available(#4843)" (#4862)
      
      This reverts commit a5b3831d.
      b0333c53
    • J
      User query can use global shared memory across resource group when available(#4843) · a5b3831d
      Jialun 提交于
      1) Global shared memory will be used if the query has run out of
      	the group shared memory.
      2) The limit of memory_spill_ratio changes to [0, INT_MAX], because
      	global shared memory can be allocated, 100% limitation is not
      	make sense.
      3) Using Atomic Operation & "Compare And Save" instead of lock to
      	get high performance.
      4) Modify the test cases according to the new rules.
      a5b3831d
    • D
      Disable large objects · 6a343b61
      David Kimura 提交于
      Large objects are currently not supported in Greenplum. Rather than deceive the
      user with a non-functional large object api, we disable them for now.
      
      We disable the large object tests in privileges regress test by using ignore
      blocks instead of commenting them out or deleting them to reduce merge
      conflicts in future postgres merges.
      Co-authored-by: NJimmy Yih <jyih@pivotal.io>
      6a343b61
  4. 14 4月, 2018 2 次提交
    • A
      Remove AppendOnlyStorage_GetUsableBlockSize(). · 0a119de3
      Ashwin Agrawal 提交于
      When the blocksize is 2MB, the function AppendOnlyStorage_GetUsableBlockSize
      would give out the wrong usable block size. The expected result is 2MB. But the
      return value of the function call would give out (2M -4). This is because the
      macro AOSmallContentHeader_MaxLength is defined as (2M -1). After rounding down
      to 4 byte aligned, the result is (2M - 4).
      
      Without the fix can encounter errors as follows: "ERROR: Used length 2097152
      greater than bufferLen 2097148 at position 8388592 in table 'xxxx'".
      
      Also removed some related, but unused macro variables, just for cleaning up
      codes related to AO storage.
      Co-authored-by: NLirong Jian <jian@hashdata.cn>
      0a119de3
    • V
      Remove FIXME in statistic computation of planner · b7abfcbf
      Venkatesh Raghavan 提交于
      Changes in 8.4 for statistics computation is comprehensive.
      Will fix any fallouts of this change as we observe them.
      Current tests show that things are kosher so removing the fix me.
      b7abfcbf
  5. 13 4月, 2018 1 次提交
    • A
      Maintain oldest xmin among distributed snapshots separately on QD · 878c7694
      Asim R P 提交于
      Commit b3f300b9 introduced the novel idea tracking oldest xmin
      among all distributed snapshots on QEs.  However, the idea is not
      applicable to QD because all distributed transactions can be found in
      ProcArray on QD.  Local oldest xmin is therefore the oldest xmin among
      all distributed snapshots on QD.  This patch fixes the maintenance of
      oldest xmin on QD by avoiding DistributedLog_AdvanceOldestXmin() and all
      the heavy-lifting that it performs.  Calling this on QD was also hitting
      the "local snapshot's xmin is older than recorded distributed
      oldestxmin" error occasionally in CI.
      878c7694
  6. 12 4月, 2018 3 次提交
    • J
      Implement NUMERIC upgrade for AOCS versions < 8.3 · 54895f54
      Jacob Champion 提交于
      8.2->8.3 upgrade of NUMERIC types was implemented for row-oriented AO
      tables, but not column-oriented. Correct that here.
      
      Store upgraded Datum data in a per-DatumStream buffer, to avoid
      "upgrading" the same data multiple times (multiple tuples may be
      pointing at the same data buffer, for example with RLE compression).
      Cache the column's base type in the DatumStreamRead struct.
      Co-authored-by: NTaylor Vesely <tvesely@pivotal.io>
      54895f54
    • B
      Revert "Fix bug that planner generates redundant motion for joins on distribution key" · 2a326e59
      Bhuvnesh Chaudhary 提交于
      This reverts commit 8b0a7fed.
      
      Due to this commit, full join queries with condition on varchar columns
      started failing due to the below error. It is expected that there is a
      relabelnode on top of varchar columns while looking up the sort
      operator, however because of the said commit we removed the relabelnode.
      
      ```sql
      create table foo(a varchar(30), b varchar(30));
      postgres=# select X.a from foo X full join (select a from foo group by 1) Y ON X.a = Y.a;
      ERROR:  could not find member 1(1043,1043) of opfamily 1994 (createplan.c:4664)
      ```
      
      Will reopen the issue #4175 which brought this patch.
      2a326e59
    • A
      Add missing #ifdef block in aset.c (#4704) · 3ddbb283
      Andreas Scherbaum 提交于
      3ddbb283
  7. 11 4月, 2018 5 次提交
    • P
      Add a GUC to limit the number of slices for a query · d716a92f
      Pengzhou Tang 提交于
      Executing a query plan containing a large number of slices may slow down
      the entire Greenplum cluster: each "n-gang" slice corresponds to a
      separate process per segment. An example of such queries is a UNION ALL
      atop several complex views. To prevent such a situation, add a GUC
      gp_max_slices and refuse to execute plans of which the number of slices
      exceed that limit.
      Signed-off-by: NJesse Zhang <sbjesse@gmail.com>
      d716a92f
    • X
      Add missing header file of gppc · adba45a9
      xiong-gang 提交于
      adba45a9
    • X
      Add GUC verify_gpfdists_cert · d66a7a1f
      xiong-gang 提交于
      This GUC determines whether curl verifies the authenticity of the
      gpfdist's certificate
      d66a7a1f
    • B
      Fix Analyze privilege issue when executed by superuser · 3c139b9f
      Bhuvnesh Chaudhary 提交于
      The patch 62aba765 from upstream fixed
      the CVE-2009-4136 (security vulnerability) with the intent to properly
      manage session-local state during execution of an index function by a
      database superuser, which in some cases allowed remote authenticated
      users to gain privileges via a table with crafted index functions.
      
      Looking into the details of the CVE-2009-4136 and related CVE-2007-6600,
      the patch should ideally have limited the scope while we calculate the
      stats on the index expressions, where we run functions to evaluate the
      expression and could potentially present a security threat.
      
      However, the patch changed the user to table owner before collecting the
      sample, due to which even if analyze was run by superuser the sample
      couldn't be collected as the table owner did not had sufficient
      privileges to access the table. With this commit, we switch back to the
      original user while collecting the sample as it does not deal with
      indexes, or function call which was the original intention of the patch.
      
      Upstream did not face the privilege issue, as it does block sampling
      instead of issuing a query.
      Signed-off-by: NSambitesh Dash <sdash@pivotal.io>
      3c139b9f
    • A
      Address GPDB_84_MERGE_FIXME in simplify_EXISTS_query() · 99450728
      Abhijit Subramanya 提交于
      This FIXME is two-fold:
      - Handling LIMIT 0
        The LIMIT is already handled in the caller,
        convert_EXISTS_sublink_to_join(): When an existential sublink contains
        an aggregate without GROUP BY or HAVING, we can safely replace it by a
        one-time TRUE/FALSE filter based on the type of sublink since the
        result of aggregate is always going to be one row even if it's input
        rows are 0.  However this assumption is incorrect when sublink
        contains LIMIT/OFFSET, such as, if the final limit count after
        applying the offset is 0.
      
      - Rules for demoting HAVING to WHERE
        previously, simplify_EXISTS_query() only disallowed demoting HAVING
        quals to WHERE, if it did not contain any aggregates. To determine the
        same, previously it used query->hasAggs, which is incorrect
        since hasAggs indicates that aggregate is present either in targetlist
        or HAVING.  This penalized the queries, wherein HAVING did not contain
        the agg but targetlist did (as demonstrated in the newly added test).
        This check is now replaced by contain_aggs_of_level().  Also, do not
        demote if HAVING contains volatile functions since they need to be
        evaluated once per group.
      Signed-off-by: NDhanashree Kashid <dkashid@pivotal.io>
      99450728
  8. 10 4月, 2018 6 次提交
    • R
      Fix setting relfrozenxid during upgrade. · aa646fe7
      Richard Guo 提交于
      Relations with external storage, as well as AO and CO
      should have InvalidTransactionId in relfrozenxid during
      upgrade.
      
      The idea here is to keep the same logic with function
      should_have_valid_relfrozenxid().
      aa646fe7
    • R
      When dumping function, append callback function attribute if needed. · d5842dc3
      Richard Guo 提交于
      Attribute 'DESCRIBE' is added by gpdb to describe
      the name of a callback function. Currently pg_dump
      does not handle this attribute.
      Co-authored-by: NPaul Guo <paulguo@gmail.com>
      d5842dc3
    • B
      Revert "Fix pushing down of quals in subqueries contains window funcs" · cc11b40e
      Bhuvnesh Chaudhary 提交于
      This reverts commit 54ee5b5c.
      
      In qual_is_pushdown_safe_set_operation it crashed while
      Assert(subquery)
      cc11b40e
    • B
      Fix pushing down of quals in subqueries contains window funcs · 54ee5b5c
      Bhuvnesh Chaudhary 提交于
      Previously, if there was a subquery contaning window functions, pushing
      down of the filters was banned. This commit fixes the issue, by pushing
      downs filters which are not on the columns projected using window
      functions.
      
      Adding relevant tests.
      
      Test case:
      After porting the fix to gpdb master, in the below case the filter `b = 1` is pushed down on
      ```
      explain select b from (select b, row_number() over (partition by b) from foo) f  where b = 1;
                                                   QUERY PLAN
      ----------------------------------------------------------------------------------------------------
       Gather Motion 3:1  (slice2; segments: 3)  (cost=0.00..1.05 rows=1 width=4)
         ->  Subquery Scan on f  (cost=0.00..1.05 rows=1 width=4)
               ->  WindowAgg  (cost=0.00..1.04 rows=1 width=4)
                     ->  Redistribute Motion 3:3  (slice1; segments: 3)  (cost=0.00..1.03 rows=1 width=4)
                           Hash Key: foo.b
                           ->  Seq Scan on foo  (cost=0.00..1.01 rows=1 width=4)
                                 Filter: b = 1
       Optimizer: legacy query optimizer
      ```
      
      Currently on master the plan is as below where the filter is not pushed down.
      ```
      explain select b from (select b, row_number() over (partition by b) from foo) f  where b = 1;
                                                   QUERY PLAN
      ----------------------------------------------------------------------------------------------------
       Gather Motion 3:1  (slice2; segments: 3)  (cost=1.04..1.07 rows=1 width=4)
         ->  Subquery Scan on f  (cost=1.04..1.07 rows=1 width=4)
               Filter: f.b = 1
               ->  WindowAgg  (cost=1.04..1.06 rows=1 width=4)
                     Partition By: foo.b
                     ->  Sort  (cost=1.04..1.04 rows=1 width=4)
                           Sort Key: foo.b
                           ->  Redistribute Motion 3:3  (slice1; segments: 3)  (cost=0.00..1.03 rows=1 width=4)
                                 Hash Key: foo.b
                                 ->  Seq Scan on foo  (cost=0.00..1.01 rows=1 width=4)
       Optimizer: legacy query optimizer
      54ee5b5c
    • B
      Add IDLE_RESOURCES_NEVER_TIME_OUT macro for magic 0 value · feb34228
      Ben Christel 提交于
      This is just refactoring. Related to (#4690).
      Co-authored-by: NBen Christel <bchristel@pivotal.io>
      Co-authored-by: NAmil Khanzada <akhanzada@pivotal.io>
      feb34228
    • A
      Remove GPDB_84_MERGE_FIXME in parse_clause.c · d4214d4e
      Abhijit Subramanya 提交于
      This commit replaces the usage of CdbOpenRelationRv() with parserOpenTable()
      for updates and deletes in setTargetList(). We cannot use the same function for
      inserts since it upgrades the lock to Exclusive mode so leave the heap_openrv()
      call intact there.
      d4214d4e
  9. 09 4月, 2018 3 次提交
    • J
      tuptoaster: ensure toast index validity · 06275c39
      Jacob Champion 提交于
      Partial backport of functionality from commit 2ef085d0 in Postgres 9.4,
      which checks to make sure that a toast table has a valid index. Prior to
      this change, toast tables with invalid indices (for example, after a
      pg_upgrade) caused confusing error messages of the form
      
          missing chunk number 0 for toast value <> in pg_toast_<>
      
      A regression test case has been added using isolation2.
      06275c39
    • J
      isolation2: add a master-and-primary utility command · 61671472
      Jacob Champion 提交于
      The '*U:' command flag will connect to the master and all primaries to
      execute its query. The intended use case is for easy testing of
      distributed catalog changes.
      61671472
    • P
      Don't pass superuser flag of SessionUserId/OuterUserId to segments · 9698d51f
      Pengzhou Tang 提交于
      GUC "is_supersuer" only provide value for SHOW to display, it's
      useless on segments. Those two flags are all designed to determine
      the value of is_superuser, so it's unnecessary to pass them to the
      segments.
      
      With this commit, another problem is resolved, GPDB used to dispatch
      a command within empty transaction and resource owner to define an
      index concurrently. However, a syscache access in superuser_arg()
      may report a SIGSEGV because CurrentResourceOwner is NULL, this commit use
      SessionUserIsSuperuser instead of superuser_arg() to avoid such error.
      9698d51f
  10. 08 4月, 2018 2 次提交
    • P
      Allow some partition related code run for pg_upgrade. (#4774) · ab129d0a
      Paul Guo 提交于
      For pg_upgrade, gpdb runs in GP_ROLE_UTILITY mode and with IsBinaryUpgrade set as true.
      In this patch we allow some partition related code run if IsBinaryUpgrade is true so that
      those partition related sql clauses, which are generated by pgdump, could recover
      previous partition schemas.
      Co-authored-by: NMax Yang <myang@pivotal.io>
      ab129d0a
    • P
      Fix bug for dropped columns in partition table in pg_dump. (#4776) · fab44240
      Paul Guo 提交于
      pg_dump creates dummy column for previously dropped column, and
      then reconstructs some column values in pg_attribute and
      finally drops the dummy column. The code does not handle the case of
      gp partition table. For gp partition table, the root table and all of its
      inherited tables should go through the same steps.
      
      Without this patch, people could see lines in pg_dump sql file similar as below with some partition
      table DDL, e.g. with table co_cr_sub_partzlib8192_1_2 in icw, run "Alter table co_cr_sub_partzlib8192_1_2 Drop column a12;".
      
      COLUMN "........pg.dropped.8........" ENCODING (compresslevel=5, compresstype=zlib, blocksize=32768)
      fab44240
  11. 07 4月, 2018 3 次提交
    • A
      Make use of transaction options dispatched by QD on QE · a6eea210
      Asim R P 提交于
      To enforce consistent isolation level within a distributed transaction, local
      transactions on QEs should assume the same isolation level as the transaction
      on QD.  This was previously achieved by dispatching isolation level and
      read-only property from QD to QE as command line options.  In case of explicit
      BEGIN, the isolation level was dispatched as flags in DtxContextInfo.  This
      patch makes it consistent such that QEs (readers as well as writers) read the
      transaction options from DtxContextInfo.
      
      The poblem with setting transaction isolation level as command line opiton is
      command line options are processed during process initialization, when a
      transaction is already open and a snapshot is taken.  Changing isolation level
      after taking a snapshot is not correct.
      
      This patch allows merging with the check for transaction_isolation GUC as it
      stands in 9.1, without any Greenplum-specific changes.
      Co-authored-by: NJacob Champion <pchampion@pivotal.io>
      a6eea210
    • A
      Move updates to SharedLocalSnapshotSlot under one lock. · ced70000
      Asim R P 提交于
      This was probably an oversight, there is no need to update "ready" field of
      SharedLocalSnapshotSlot object separately.
      ced70000
    • A
      Use ereportif() where logging is predicated by a debug GUC. · 4351a1e1
      Asim R P 提交于
      No change in functionality here.  The ereportif() macro avoids ereport()
      invocation if the preficate is not true.
      
      Fixed indentation in a couple of places on the way.
      4351a1e1
  12. 05 4月, 2018 3 次提交
    • D
      Fix fallback test in gporca.sql · 34878131
      Dhanashree Kashid 提交于
      This test was added to check the logging of ORCA fall-back messages. The
      query contains CUBE grouping extension which is currently not supported
      by ORCA causing ORCA to fall back to planner with following log
      messages:
      
      LOG:  NOTICE,"Feature not supported by the Pivotal Query Optimizer:
      Cube",
      LOG:  Planner produced plan :0
      
      The planner generated plan contains a Shared Scan node.  During
      execution of this, sometimes, there is an extra log message generated
      indicating that Shared Scan writer is waiting for an acknowledgement
      from Shared Scan readers:
      
      LOG: SISC WRITER (shareid=0, slice=1): notify still wait for an answer,
      errno 4
      
      The query returns successfully however this intermittently generated log
      message causes this test to fail.
      This commit fixes the flake by converting this to an EXPLAIN test, which
      is sufficient to demonstrate the fall back logging.
      34878131
    • J
      Fix issue converting server encoding to client encoding · 59cb001b
      Jimmy Yih 提交于
      This was found when server encoding was WIN1251 but the client
      encoding was UTF8. The conversion is valid for these character sets
      but the client would receive an "invalid byte sequence for encoding"
      error. We fix the issue by calling the correct conversion function. It
      was switched up during a postgres backport last year.
      
      Thanks to Heikki for debugging the issue.
      59cb001b
    • J
      Metadata tracking for ALTER ROLE IN DATABASE · adf4a029
      Jimmy Yih 提交于
      There was a GPDB_90_MERGE_FIXME regarding what we should report for
      ALTER ROLE IN DATABASE in pg_stat_last_shoperation table for metadata
      tracking. We should always report that the role object was altered if
      coming from ALTER ROLE code path. To do this, we just check if the
      role oid is valid because the call to AlterSetting from ALTER DATABASE
      uses InvalidOid for role oid. While we're at it, we might as well add
      some tests for pg_stat_last_shoperation.
      adf4a029
  13. 03 4月, 2018 1 次提交
    • A
      pg_dump: simplify EXTERNAL LOG ERRORS logic · aaecc6da
      Adam Lee 提交于
      Since error tables are not supported, we only need to identify error
      logging or not, which is if the error table OID is not NULL under
      version 6.0.
      
      On 5.X, the error table OID is the table itself to log errors into file,
      is another to into the error table. Not NULL means logging errors.
      aaecc6da