1. 12 5月, 2020 1 次提交
    • H
      Limit DPE stats to groups with unresolved partition selectors (#9988) · dddd8366
      Hans Zeller 提交于
      DPE stats are computed when we have a dynamic partition selector that's
      applied on another child of a join. The current code continues to use
      DPE stats even for the common ancestor join and nodes above it, but
      those nodes aren't affected by the partition selector.
      
      Regular Memo groups pick the best expression among several to compute
      stats, which makes row count estimates more reliable. We don't have
      that luxury with DPE stats, therefore they are often less reliable.
      
      By minimizing the places where we use DPE stats, we should overall get
      more reliable row count estimates with DPE stats enabled.
      
      The fix also ignores DPE stats with row counts greater than the group
      stats. Partition selectors eliminate certain partitions, therefore
      it is impossible for them to increase the row count.
      dddd8366
  2. 09 5月, 2020 11 次提交
  3. 08 5月, 2020 2 次提交
    • Z
      Fix possible crash in COPY FORM on QEs · e9c00b80
      Zhenghua Lyu 提交于
      Target partitions need new ResultRelInfos and override previous
      estate->es_result_relation_info in NextCopyFromExecute(). The
      new ResultRelInfo may leave its resultSlot as NULL. If sreh is
      on, the parsing errors will be caught and loop back to parse
      another row; however, the estate->es_result_relation_info was
      already changed. This can cause crash.
      
      Reproduce:
      
      ```sql
      CREATE TABLE partdisttest(id INT, t TIMESTAMP, d VARCHAR(4))
      DISTRIBUTED BY (id)
      PARTITION BY RANGE (t)
      (
        PARTITION p2020 START ('2020-01-01'::TIMESTAMP) END ('2021-01-01'::TIMESTAMP),
        DEFAULT PARTITION extra
      );
      
      COPY partdisttest FROM STDIN LOG ERRORS SEGMENT REJECT LIMIT 2;
      1	'2020-04-15'	abcde
      1	'2020-04-15'	abc
      \.
      ```
      Authored-by: Nggbq <taos.alias@outlook.com>
      e9c00b80
    • P
      Fix a spinlock leak for fault injector · d1b10d64
      Pengzhou Tang 提交于
      This is a backport from master 2a7b2bf6
      d1b10d64
  4. 07 5月, 2020 1 次提交
  5. 06 5月, 2020 1 次提交
    • F
      6X Backport: Enable parallel writes for Foreign Data Wrappers · 86f6c666
      Francisco Guerrero 提交于
      This commit enables parallel writes for Foreign Data Wrapper. This
      feature is currently missing from the FDW framework, whilst parallel
      scans are supported, parallel writes are missing. FDW parallel writes
      are analogous to writing to writable external tables that run on all
      segments.
      
      One caveat is that in the external table framework, writable tables
      support a distribution policy:
      
          CREATE WRITABLE EXTERNAL TABLE foo (id int)
          LOCATION ('....')
          FORMAT 'CSV'
          DISTRIBUTED BY (id);
      
      In foreign tables, the distribution policy cannot be defined during the
      table definition, so we assume random distribution for all foreign
      tables.
      
      Parallel writes are enabled when the foreign table's exec_location is
      set to FTEXECLOCATION_ALL_SEGMENTS only. For foreign tables that run on
      master or any segment, the current policy behavior remains.
      86f6c666
  6. 01 5月, 2020 3 次提交
  7. 29 4月, 2020 3 次提交
  8. 28 4月, 2020 6 次提交
    • P
      Fix a bug that reader gang always fail due to missing writer gang. (#9828) · 7a665560
      Paul Guo 提交于
      The reason is that new created reader gang would fail on QE due to missing
      writer gang process in locking code, and retry would fail again with the same
      reason, since the cached writer gang is still used because QD does not know &
      check the real libpq network status. See below for the repro case.
      
      Fixing this by checking the error message and then reset all gangs if seeing
      the error message, similar to the code logic that checks the startup/recovery
      message in gang create function. We could have other fixes, e.g. checking the
      writer gang network status, etc but those fixes seem to be ugly after trying.
      
      create table t1(f1 int, f2 text);
      <kill -9 one idle QE>
      
      insert into t1 values(2),(1),(5);
      ERROR:  failed to acquire resources on one or more segments
      DETAIL:  FATAL:  reader could not find writer proc entry, lock [0,1260] AccessShareLock 0 (lock.c:874)
       (seg0 192.168.235.128:7002)
      
      insert into t1 values(2),(1),(5);
       ERROR:  failed to acquire resources on one or more segments
       DETAIL:  FATAL:  reader could not find writer proc entry, lock [0,1260] AccessShareLock 0 (lock.c:874)
        (seg0 192.168.235.128:7002)
      
      <-- Above query fails again.
      
      Cherry-picked from 24f16417 and a0a5b4d5
      7a665560
    • P
      Let Fts tolerate the in-progress 'starting up' case on primary nodes. · 5222ad86
      Paul Guo 提交于
      commit d453a4aa implemented that for the crash
      recovery case (not marking the node down and then not promoting the mirror). It
      seems that we should do that for the usual "starting up" case also(i.e.
      CAC_STARTUP), besides for the existing "in recovery mode" case (i.e.
      CAC_RECOVERY).
      
      We've seen that fts promotes the "starting up" primary during isolation2
      testing due to 'pg_ctl restart'. In this patch we check recovery progress for
      both CAC_STARTUP an CAC_RECOVERY during fts probe and thus can avoid this.
      Reviewed-by: NAshwin Agrawal <aagrawal@pivotal.io>
      
      cherry-picked from d71b3afd
      
      On master the commit message was eliminated by mistake. Added back on gpdb6.
      5222ad86
    • P
      Remove forceEos mechanism for TCP interconnect · 7cf0ac40
      Pengzhou Tang 提交于
      In TCP interconnect, the sender used to force an EOS messages to the
      receiver in two cases:
      1. cancelUnfinished is true in mppExecutorFinishup.
      2. an error occurs.
      
      For case1, the comment says: to finish a cursor, the QD used to send
      a cancel to the QEs, QEs then set the cancelUnfinished flag and did
      a normal executor finish up. We now use QueryFinishPending mechanism
      to stop a cursor, so case1 logic is invalid for a long time.
      
      For case2, the purpose is: when an error occurs, we force an EOS to
      the receiver so the receiver didn't report an interconnect error and
      QD then will check the dispatch results and report the errors in the
      QEs. From the view of interconnect, we have selectedd to the end of
      the query and no error in the interconnect, this logic has two
      problems:
      1. it doesn't work for initplan, initplan will not check the dispatch
      results and throw the errors, so when an error occurs in the QEs for
      the initplan, the QD cannot notice that.
      2. it doesn't work for cursors, for example:
         DECLARE c1 cursor for select i from t1 where i / 0 = 1;
         FETCH all from c1;
         FETCH all from c1;
      All FETCH commands don't report errors which is not expected.
      
      This commit removed the forceEos mechanism, for the case2, the
      receiver will report an interconnect error without forceEos, this is
      ok because when multiple errors reports from QEs, the QD is inclined
      to report non-interconnect error.
      7cf0ac40
    • P
      Remove redundant 'hasError' flag in TeardownTCPInterconnect · d093c024
      Pengzhou Tang 提交于
      This flag is duplicated with 'forceEOS', 'forceEOS' can also tell
      whether errors occur or not.
      d093c024
    • P
      Fix a race condition in flushBuffer · 3e1cc863
      Pengzhou Tang 提交于
      flushBuffer() is used to send packets through TCP interconnect, before
      sending, it first check whether receiver stopped or teared down the
      interconnect, however, there is window between checking and sending, the
      receiver may tear down the interconnect and close the peer, so send()
      will report an error, to resolve this, we recheck whether the receiver
      stopped or teared down the interconnect in this window and don't error
      out in that case.
      Reviewed-by: NJinbao Chen <jinchen@pivotal.io>
      Reviewed-by: NHao Wu <hawu@pivotal.io>
      3e1cc863
    • P
      Fix interconnect hung issue · 7c90c04f
      Pengzhou Tang 提交于
      We hit interconnect hung issue many times in many cases, all have
      the same pattern: the downstream interconnect motion senders keep
      sending the tuples and they are blind to the fact that upstream
      nodes have finished and quitted the execution earlier, the QD
      then get enough tuples and wait all QEs to quit which cause a
      deadlock.
      
      Many nodes may quit execution earlier, eg, LIMIT, HashJoin, Nest
      Loop, to resolve the hung issue, they need to stop the interconnect
      stream explicitly by calling ExecSquelchNode(), however, we cannot
      do that for rescan cases in which data might lose, eg, commit
      2c011ce4. For rescan cases, we tried using QueryFinishPending to
      stop the senders in commit 02213a73 and let senders check this
      flag and quit, that commit has its own problem, firstly, QueryFini
      shPending can only set by QD, it doesn't work for INSERT or UPDATE
      cases, secondly, that commit only let the senders detect the flag
      and quit the loop in a rude way (without sending the EOS to its
      receiver), the receiver may still be stuck inreceiving tuples.
      
      This commit revert the QueryFinishPending method firstly.
      
      To resolve the hung issue, we move TeardownInterconnect to the
      ahead of cdbdisp_checkDispatchResult so it guarantees to stop
      the interconnect stream before waiting and checking the status
      of QEs.
      
      For UDPIFC, TeardownInterconnect() remove the ic entries, any
      packets for this interconnect context will be treated as 'past'
      packets and be acked with STOP flag.
      
      For TCP, TeardownInterconnect() close all connection with its
      children, the children will treat any readable data in the
      connection as a STOP message include the closure operation.
      
      A test case is not included, both commit 2c011ce4 and 02213a73
      contain one.
      7c90c04f
  9. 27 4月, 2020 1 次提交
    • P
      Fix a bug that two-phase sub-transaction is considered as one-phase. · 48ffabce
      Paul Guo 提交于
      QD backend should not forget whether a sub transaction performed writes
      
      QD backend process can avoid two-phase commit overhead if it knows that no QEs
      involved in this transaction or any of its sub transactions performed any
      writes. Previously, if a sub transaction performed write on one or more QEs, it
      was remembered in that sub transaction's global state. However, the sub
      transaction state was lost after sub transaction commit. That resulted in QD
      not performing two-phase commit at the end of top transaction.
      
      In fact, regardless of the transaction nesting level, we only need to remember
      whether a write was performed by a sub transaction. Therefore, use a backend
      global variable, instead of current transaction state to record this
      information.
      Reviewed-by: NGang Xiong <gxiong@pivotal.io>
      Reviewed-by: NHao Wu <gfphoenix78@gmail.com>
      Reviewed-by: NAsim R P <apraveen@pivotal.io>
      48ffabce
  10. 24 4月, 2020 1 次提交
  11. 23 4月, 2020 1 次提交
  12. 21 4月, 2020 1 次提交
    • H
      Fix getDtxCheckPointInfo to contain all committed transactions (#9942) · 05913102
      Hao Wu 提交于
      Half committed transactions in shmCommittedGxactArray are omitted.
      The bug could cause data loss/inconsistency. If transaction T1
      failed to commit prepared for some reasons, and the transaction T1
      has been committed on the master and other segments, but the transaction
      T1 isn't appended in the checkpoint record. So the DTX recovery
      can't retrieve the transaction and run recovery-commit-prepared,
      and the prepared transactions on the segment are aborted.
      Co-authored-by: NGang Xiong <gxiong@pivotal.io>
      05913102
  13. 20 4月, 2020 5 次提交
    • Z
      Improve efficiency in pg_lock_status() · 66e7f812
      Zhenghua Lyu 提交于
      Allocate memory of CdbPgResults.pg_results with palloc0() instead
      of calloc(), and free the memory afer use.
      
      The CdbPgResults.pg_results array that is returned from various dispatch
      functions is allocated by cdbdisp_returnResults() via calloc(), but in
      most cases the memory is not free()-ed after use.
      
      To avoid memory leak, the array is now allocated with palloc0() and
      recycled with pfree().
      
      Track which row and which result set is being processed in function context in pg_lock_status(), 
      so that an ineffient inner loop can be eliminated.
      
      Author: Fang Zheng <zhengfang.xjtu@gmail.com>
      66e7f812
    • S
      Do not push Volatile funcs below aggs · 89308890
      Sambitesh Dash 提交于
      Consider the scenario below
      
      ```
      create table tenk1 (c1 int, ten int);
      create temp sequence ts1;
      explain select * from (select distinct ten from tenk1) ss where ten < 10 + nextval('ts1') order by 1;
      ```
      
      The filter outside the subquery is a candidate to be pushed below the
      'distinct' in the sub-query.  But since 'nextval' is a volatile function, we
      should not push it.
      
      Volatile functions give different results with each execution. We don't want
      aggs to use result of a volatile function before it is necessary. We do it for
      all aggs - DISTINCT and GROUP BY.
      
      Also see commit 6327f25d.
      89308890
    • H
      Fix memory leak and bug in checkpointer process (#9766) · fa07e427
      Hao Wu 提交于
      1. The CurrentMemoryContext in CreateCheckPoint is long-live
      memory context owned by the checkpointer process. The memory context
      is reset only if the error occurs. So memory leak will lead to
      huge memory leak in the OS.
      2. The prepared transactions are not appended as an extension in
      the checkpoint wal record, which introduces a bug. It occurs:
        1) T1 does some DML and is prepared.
        2) T2 runs a checkpoint.
        3) The seg0 crashes before the transaction T1 successfully runs `COMMIT PREPARE`, but
           all other segments have successfully committed.
        4) When running local recovery, seg0 doesn't know T1 is prepared
           and needs to be committed again.
      This is different from the master branch. Master uses files to record
      whether there exists any prepared transaction and what them are.
      Reviewed-by: NNing Yu <nyu@pivotal.io>
      Reviewed-by: NGang Xiong <gxiong@pivotal.io>
      fa07e427
    • X
      Fix a bug when setting DistributedLogShared->oldestXmin · e246d777
      xiong-gang 提交于
      The shared oldestXmin (DistributedLogShared->oldestXmin) may be updated
      concurrently. It should be set to a higher value, because a higher xmin
      can belong to another distributed log segment, its older segments might
      already be truncated.
      
      For Example: txA and txB call DistributedLog_AdvanceOldestXmin concurrently.
      
      ```
      txA and txB: both hold shared DistributedLogTruncateLock.
      
      txA: set the DistributedLogShared->oldestXmin to XminA. TransactionIdToSegment(XminA) = 0009
      
      txB: set the DistributedLogShared->oldestXmin to XminB. TransactionIdToSegment(XminB) = 0008
      
      txA: truncate segment 0008, 0007...
      ```
      
      After that, DistributedLogShared->oldestXmin == XminB, it is on removed
      segment 0008. Subsequent GetSnapshotData() calls will be failed because SimpleLruReadPage will error out.
      Co-authored-by: Ndh-cloud <60729713+dh-cloud@users.noreply.github.com>
      e246d777
    • X
      Fix CTAS 'with no data' bug · a006571b
      xiong-gang 提交于
      As reported in issue #9790, 'CTAS with no data' statement doesn't handle WITH
      clause, the options in WITH clause should be added in 'pg_attribute_encoding'.
      a006571b
  14. 18 4月, 2020 1 次提交
    • Z
      Fix plan when segmentgeneral union all general locus. · ca560132
      Zhenghua Lyu 提交于
      Previously, planner cannot generate plan when a replicated
      table union all a general locus scan. A typical case is:
      
        select a from t_replicate_table
        union all
        select * from generate_series(1, 10);
      
      The root cause is in the function `set_append_path_locus`
      it deduces the whole append path's locus to be segmentgeneral.
      This is reasonable. However, in the function `cdbpath_create_motion_path`
      it fails to handle two issues in the case:
        * segmentgeneral locus to segmentgeneral locus
        * general locus to segmentgeneral locus
      And the both above locus change does not need motion in fact.
      
      This commit fixes this by:
        1. add a check at the very begining of `cdbpath_create_motion_path`
           that if the subpath's locus is the same as target locus, just return
        2. add logic to handle general locus to segmentgeneral locus, just return
      ca560132
  15. 15 4月, 2020 1 次提交
    • S
      Speed up stats derivation for large number of disjunction in ORCA · 085952c1
      Shreedhar Hardikar 提交于
      This bug is particularly evident with queries containing a large array
      IN clause, e.g "a IN (1, 3, 5, ...)".
      
      As a first step to improve optimization times for such queries, this
      commit reduces unnecessary re-allocation of histogram buckets during the
      merging of statistics of disjunctive predicates.
      
      It improves the performance of the target query with 7000 elements in
      the array comparison by around 50%.
      085952c1
  16. 13 4月, 2020 1 次提交
    • Z
      Add check for indexpath when bring_to_singleQE. · 2fbc274b
      Zhenghua Lyu 提交于
      Previously, in function bring_to_singleQE,
      it depends on the path->param_info field to determine if the path
      can be taken into consideration since we cannot pass params across
      motion node. But this is not enough, for example, an index path's
      param_info field might be null, but its orderbyclauses refs some
      outer params. This commit fixes the issue by adding more check
      for indexpath.
      
      See Github Issue: https://github.com/greenplum-db/gpdb/issues/9733
      for details.
      2fbc274b