1. 28 4月, 2020 5 次提交
  2. 27 4月, 2020 4 次提交
    • H
      Stop "normalizing" the join tree of every query. · 0f97cad8
      Heikki Linnakangas 提交于
      Some of the GPDB-added subquery pull-up operations relied on the join
      tree having specific form. But it's straightforward to teach those
      operations to deal with arbitarary FromExprs and JoinExprs in the join
      tree. Do that, and remove the join tree normalization step.
      
      The reason I'm hacking on this right now is that on the 'merge' branch,
      where we're working on merging with PostgreSQL v12, the normalization was
      messing with the new SRF-in-targetlist processing. It probably would've
      been easy to fix, but if we can get rid of the normalization step
      altogether, that's even better.
      Reviewed-by: NRichard Guo <riguo@pivotal.io>
      Reviewed-by: NYandong Yao <yyao@pivotal.io>
      0f97cad8
    • P
      Fix a race condition in flushBuffer · 51c1bf91
      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>
      51c1bf91
    • P
      Fix a bug that two-phase sub-transaction is considered as one-phase. · 200a083e
      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>
      200a083e
    • P
      Enlarge timeout in isolation2:pg_ctl UDF (#9991) · 934d87c6
      Paul Guo 提交于
      Currently this UDF might report a false positive if the node is still starting
      up after timeout since currently pg_ctl returns 0 for this case. This behavior
      is changed in upstream with the below patch:
      
      commit f13ea95f
      Author: Tom Lane <tgl@sss.pgh.pa.us>
      Date:   Wed Jun 28 17:31:24 2017 -0400
      
          Change pg_ctl to detect server-ready by watching status in postmaster.pid.
      
      We've seen some test flakiness due to this issue since pg_ctl restart needs
      more time sometimes on pipeline (by default pg_ctl timeout is 60 seconds).
      Yesterday I found on a hang job that a primary needs ~ 4 minutes to get the
      recovery finished during 'pg_ctl restart' (It's test
      ao_same_trans_truncate_crash which enables fsync. Even it launches a checkpoint
      before pg_ctl restart, pg_ctl restarts still needs a lot of time).
      
      Enlarge the timeout of pg_ctl to 600 seconds now and add a pg_ctl stdout
      checking before returning OK in the UDF (this check could be removed after PG
      12 merge finishes so I added a FIXME there).
      
      Here is the output of the pg_ctl experiment:
      
      $ pg_ctl -l postmaster.log -D /data/gpdb7/gpAux/gpdemo/datadirs/dbfast1/demoDataDir0 -w -m immediate restart -t 1
      waiting for server to shut down.... done
      server stopped
      waiting for server to start.... stopped waiting
      server is still starting up
      $ echo $?
      0
      Reviewed-by: NAsim R P <apraveen@pivotal.io>
      934d87c6
  3. 26 4月, 2020 2 次提交
    • X
      Fix ALTER DATABASE SET ... FROM CURRENT · 5a855614
      xiong-gang 提交于
      ALTER DATABASE SET ... FROM CURRENT dispatches incorrect statement to the
      segments. Reported in https://github.com/greenplum-db/gpdb/issues/9823
      5a855614
    • Z
      Correct and speed up some isolation2 test cases. · 1e68270b
      Zhenghua Lyu 提交于
      * Enable GDD for test concurrent update table with varying length type.
      
      This test case of "test concurrent update table with varying length type"
      in isolation2/concurrent_update is introduced by the commit 8ae681e5. It
      assumes to test evalPlanQual's logic so GDD has to be enabled.
      
      The mistake happens because when commit 8ae681e5 goes in, GDD is default
      enabled. Later commit 29e7f102 set GDD defaultly disabled but did not
      handle this test case correctly
      
      This commit fixes this by moving the case in the env with GDD enabled.
      
      * Make concurrent_update* test concise.
      
      Previously, concurrent_update_epq and concurrent_update_distkey
      have some cases for AO/CO table, and some cases running without GDD.
      That does not make much sense because:
        * for AO/CO table, it always holds ExclusiveLock on the table no
          matter GDD is enabled or disabled
        * without GDD, it always holds ExclusiveLock on the table
      
      I believe for the above two naive cases, just checking lockmode on
      the table should be enough, and isolation2/lockmodes test case has
      already covered this.
      
      This commit removes such cases concurrent_update_epq and
      concurrent_update_distkey. After this, the two scripts can be
      merged into concurrent_update because they all need GDD to be
      enabled and this commit merges them into one test script.
      
      After this, we can move the only remaining test concurrent_update
      to the gdd suites to save time of extra restarting cluster twice.
      
      * Restart the whole cluster for GDD since QEs will use the GUC.
      
      Previous commit a4b2fea3 uses a skill that only restart
      master node's postmaster to avoid restart the whole Greenplum
      cluster so that saves some time for pipeline test. The skill
      works under the assumption that gp_enable_global_deadlock_detector
      only needs on master. But actually, in QEs, we also have code
      that checking this GUC: ExecDelete and XactLockTableWait. So
      we'd better restart the whole cluster to make the GUC in QEs
      also changed.
      
      I find this issue when I am trying to use the skill of commit
      a4b2fea3 for the isolation2 test case `concurrent_update`,
      that case was introduced by the commit 39fbfe96 and that
      commit adds some code running on QEs but need the GUC
      gp_enable_global_deadlock_detector.
      1e68270b
  4. 25 4月, 2020 3 次提交
    • M
      docs - updates for PostGIS 2.5.4 (#9989) · fcc3f377
      Mel Kiyama 提交于
      * docs - updates for PostGIS 2.5.4
      
      -Updated version 2.1.5 -> 2.5.4
      -Updated references to PostGIS topics
      -Removed geometry and geography limitation
      -Added enhancement for postgis_manager.sh script - support installing PostGIS in a non-default schema
      -Added examples of updating postgis.enable_outdb_rasters and postgis.gdal_enabled_drivers for a GPDB session
      -Remove ANALYZE limitation for user-defined data types
      -Added  _postgis_index_extent function is not supported
      -Added note: <-> operator returns centroid/centroid distance
      -changed name of PostGIS aggregate function ST_MemCollect to ST_Collect
      
      * docs - fix typo
      
      * docs - minor changes to topic hierarchy.
      
      * docs - one more minor hierarchy change.
      
      * docs - update external URLs
      
      * docs - fix typo, minor cleanup
      
      * docs - found strange character
      fcc3f377
    • M
      docs - add Note about host requirements to segment mirror overview (#9971) · 709c0d9f
      Mel Kiyama 提交于
      * docs - add Note about host requirements to segment mirror overview
      
      -also edited some related topics and changed some links
      -fixed a bad link to the gpcheck utiltiy (removed in GPDB 6)
      
      * docs - minor edit
      Co-authored-by: NDavid Yozie <dyozie@pivotal.io>
      709c0d9f
    • M
      docs - add information about view dependency (#9952) · 05d83b31
      Mel Kiyama 提交于
      * docs - add information about view dependency
      
      -example queries to find dependency information
      -catalog table information
      -best practices
      
      * docs - minor edits and fixes.
      
      * docs - fix minor typo.
      
      * docs - another minor edit.
      05d83b31
  5. 24 4月, 2020 7 次提交
  6. 23 4月, 2020 5 次提交
    • P
      Remove forceEos mechanism for TCP interconnect · 041d9399
      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.
      041d9399
    • H
      Eliminating alien nodes on master. (#9938) · 874939f4
      Hubert Zhang 提交于
      GPDB support to eliminate alien nodes for QEs, but master still
      needs to run ExecInitNode for all the plan nodes. This will
      introduce unnecessary CPU cost on master.
      
      Ideally master only needs to ExecInitNode for all nodes on top of the
      first gather motion. Note that there are some exceptions we skip the
      optimization: explain query, explain analzye query, cursor case and
      initplan case.
      Reviewed-by: NNing Yu <nyu@pivotal.io>
      Reviewed-by: NHeikki Linnakangas <hlinnakangas@pivotal.io>
      874939f4
    • A
      fix memory leak when creating index · c1f122e5
      aqktjcm 提交于
      Memory pointed to by tuple was not released in time. The process of creating index will crash if the count of table's rows is big enough.
      This pull request is to solve the issue #9676.
      c1f122e5
    • L
      Docs - highlight vacuum after gpexpand (#9944) · f656b50e
      Lena Hunter 提交于
      * clarifying pg_upgrade note
      
      * adding VACUUM note for gpexpand
      
      * gpexpand edits
      
      * small edit
      
      * edits after review
      f656b50e
    • L
      docs - make finding upgrade & migration info easier (#9927) · 959364a1
      Lisa Owen 提交于
      - add landing page for upgrade (6->6 upgrade, 4/5->6 migration)
      - nest in subnav
      959364a1
  7. 22 4月, 2020 3 次提交
    • H
      Fix two subquery bugs. · a7df6f9b
      Heikki Linnakangas 提交于
      1. If a join qual in the subquery contains references to the outer query,
      it must be evaluated in the the same slice as the outer-query. Otherwise
      the outer variables won't be available where they're evaluated. Teach
      cdbpath_motion_for_join() to take that into account.
      
      2. After fixing the first bug, one of the new test queries was still
      failing with a runtime error:
      
      SELECT (SELECT join_tab1.i - join_tab2.i
              FROM join_tab1, join_tab2 WHERE join_tab1.i = join_tab2.i and out1.i = out2.i
              LIMIT 1) as x
      FROM join_tab1 out1, join_tab2 out2;
      ERROR:  cannot execute squelched plan node of type: 203 (execProcnode.c:948)  (seg0 slice4 127.0.0.1:40000 pid=10725) (execProcnode.c:948)
      
      If a Limit is used in a subquery, we better not squelch it if we expect the
      subquery to be rescanned.
      
      The first bug was introduced in the refactoring of subquery processing on
      'master' branch, but the second bug is present in 6X_STABLE, too.
      Reviewed-by: NZhenghua Lyu <zlv@pivotal.io>
      a7df6f9b
    • Z
      Fix some potential bugs that not update size results. · bff12f77
      Zhenghua Lyu 提交于
      At several places we just invoke the function add_size
      but does not assign its return value to any variable.
      The result is just discarded. This commit fixes this.
      bff12f77
    • B
      Add arm regression test badge into README · b85760a0
      bzhaoopenstack 提交于
      I have setup a periodic task in openlab env for gpdb.
      It runs installcheck-world per UTC12:00 every day.
      
      We can check the status with link
      
      https://status.openlabtesting.org/builds/builds?project=greenplum-db%2Fgpdb&job_name=gpdb-installcheck-world-tests-on-arm64
      
      And get the necessary log in each build.
      Also that build definition could be changed anytime. You guys can tell me
      the changes you want. Thanks
      b85760a0
  8. 21 4月, 2020 2 次提交
    • H
      Fix getDtxCheckPointInfo to contain all committed transactions (#9940) · a8eb4dc1
      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>
      a8eb4dc1
    • W
      Change random_page_access default value to 4 · 9c466e58
      Weinan WANG 提交于
      In upstream, random_page_access default value is always 4. For some unknown reason, we bump up to 100, which impacts bitmap index, index, and index-only scan cost different. After some performance tests in ao and aoco table, the diminishment of the value also can leverage these table scans' plans.
      9c466e58
  9. 20 4月, 2020 5 次提交
    • D
      Fix zero plan_node_id for BitmapOr/And in ORCA · 53a0b781
      Denis Smirnov 提交于
      According to plannode.h "plan_node_id" should be unique across
      entire final plan tree. But ORCA DXL to PlanStatement translator
      returns uninitialized zero values for BitmapOr and BitmapAnd nodes.
      This behaviour differs from Postgres planner and from all other
      node translations in this class. It was fixed.
      53a0b781
    • S
      Do not push Volatile funcs below aggs · 885ca8a9
      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.
      885ca8a9
    • H
      Use a unicast IP address for interconnection (#9696) · 790c7bac
      Hao Wu 提交于
      * Use a unicast IP address for interconnection on the primary
      
      Currently, interconnect/UDP always binds the wildcard address to
      the socket, which makes all QEs on the same node share the same
      port space(up to 64k). For dense deployment, the UDP port could run
      out, even if there are multiple IP address.
      To increase the total number of available ports for QEs on a node,
      we bind a single/unicast IP address to the socket for interconnect/UDP,
      instead of the wildcard address. So segments with different IP address
      have different port space.
      To fully utilize this patch to alleviate running out of port, it's
      better to assign different ADDRESS(gp_segment_configuration.address) to
      different segment, although it's not mandatory.
      
      Note: QD/mirror uses the primary's address value in
      gp_segment_configuration as the destination IP to connect to the
      primary.  So the primary returns the ADDRESS as its local address
      by calling `getsockname()`.
      
      * Fix the origin of the source IP address for backends
      
      The destination IP address uses the listenerAddr of the parent slice.
      But the source IP address to bind is difficult. Because it's not
      stored on the segment, and the slice table is sent to the QEs after
      they had bound the address and port. The origin of the source
      IP address for different roles is different:
      1. QD : by calling `cdbcomponent_getComponentInfo()`
      2. QE on master: by qdHostname dispatched by QD
      3. QE on segment: by the local address for QE of the TCP connection
      790c7bac
    • P
      Fix a bug that reader gang always fail due to missing writer gang. (#9828) · 24f16417
      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.
      
      The patch removes useless function GangOK() - this is not relevant of this fix
      though.
      Reviewed-by: NPengzhou Tang <ptang@pivotal.io>
      Reviewed-by: NAsim R P <apraveen@pivotal.io>
      24f16417
    • P
      Fix flaky uao_crash_compaction_row test · b00916e6
      Pengzhou Tang 提交于
      This test creates an AO table and inserts data on all segments, then
      it deletes some data on seg0 and seg1 and do a vacuum on the AO
      table. When doing vacuum, it suspends the QE in seg0 at starting
      doing the post vacuum cleanup, then crash the seg0 and finally do
      the post crash validation checks using gp_toolkit.__gp_aoseg(), this
      function will check all aoseg info on all segments.
      
      The VACUUM process on seg1 is in an uncertain state, it might have
      finished the post cleanup which is expected or hasn't started yet,
      so the aoseg info in seg1 is uncertain too.
      
      To resolve this, this commit added a new injector on the point all
      post vacuum cleanup are committed and validate the aoseg info after
      the vacuum process on seg1 reached this point.
      b00916e6
  10. 18 4月, 2020 4 次提交