1. 27 4月, 2020 1 次提交
    • P
      Enlarge timeout in isolation2:pg_ctl UDF (#9991) · 994cc7ce
      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>
      
      Cherry-picked from 934d87c6
      994cc7ce
  2. 25 4月, 2020 3 次提交
    • M
      docs - updates for PostGIS 2.5.4 (#9989) · 591d1a59
      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
      591d1a59
    • M
      docs - add Note about host requirements to segment mirror overview (#9971) · 1a959ff4
      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>
      1a959ff4
    • M
      docs - add information about view dependency (#9952) · f373c526
      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.
      f373c526
  3. 24 4月, 2020 2 次提交
  4. 23 4月, 2020 3 次提交
  5. 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
  6. 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
  7. 18 4月, 2020 5 次提交
    • 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
    • D
      Docs - fix some broken links · f166464d
      David Yozie 提交于
      f166464d
    • B
      Handle disabling of bitmap and bpchar_pattern_ops indexes in pg_upgrade · 536e7860
      Bhuvnesh Chaudhary 提交于
      In GPDB6, the access method oid is 7013 whereas it's 3013 in GPDB5.
      Until now, the bitmap indexes were not marked invalid as the query to
      update the catalog relied on incorrect OID.  This commit removes the
      hardcoded oid, and instead uses a catalog query.
      
      Also, GPDB uses the GUC allow_system_table_mods to allow updates on
      catalog tables. While disabling the indexes with bpchar_pattern_ops
      indexes, the GUC was not set and the query used to fail. This commit
      fixes it too.
      
      Since upgraded target master database is copied to segments, reset the
      index status to valid where required. The queries to ensure that all the
      objects have been migrated from old segment to new segments excludes
      invalid objects and will complain that index has not been migrated. The
      index will be marked invalid during upgrade of the segment.
      536e7860
    • D
      Docs - Clarify NOWAIT behavior/expectations with SELECT FOR UPDATE (#9936) · 5b84fc86
      David Yozie 提交于
      * Add more detail to NOWAIT behavior; standardize gdd note
      
      * Standardize gdd wording around SELECT FOR UPDATE
      
      * Remove NOWAIT example from lock_wait description
      
      * Standardize gdd note in LOCK syntax
      
      * Revert "Standardize gdd note in LOCK syntax"
      
      This reverts commit 53c51bd2f9871b097a74e25378d9c0a2d86825f9.
      
      * Revert "Standardize gdd wording around SELECT FOR UPDATE"
      
      This reverts commit 0dbd3845bb7c2300d3edd5053452800770378477.
      
      * More edits to NOWAIT wording
      
      * Docs - add MADlib 1.17 to compatibility table
      
      * Revert "Docs - add MADlib 1.17 to compatibility table"
      
      This reverts commit d4470e5c1bba81cf06c4110e378760616233064c.
      
      * Edit from Mel
      
      * Update GDD note to exclude SELECT FOR UPDATE affect, and emphasize table-level lock
      5b84fc86
    • L
  8. 16 4月, 2020 4 次提交
  9. 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
  10. 14 4月, 2020 3 次提交
  11. 13 4月, 2020 2 次提交
  12. 11 4月, 2020 2 次提交
    • S
      Handle opfamilies/opclasses for distribution in ORCA · 5e04eb14
      Shreedhar Hardikar 提交于
      GPDB 6 introduced a mechanism to distribute table tables on columns
      using a custom hash opclass, instead of using cdbhash. Before this
      commit, ORCA would ignore the distribution opclass, but ensuring the
      translator would only allow queries in which all tables were distributed
      by either their default or default "legacy" opclasses.
      
      However, in case of tables distributed by legacy or default opclasses,
      but joined using a non-default opclass operator, ORCA would produce an
      incorrect plan, giving wrong results.
      
      This commit fixes that bug by introducing support for distributed tables
      using non-default opfamilies/opclasses. But, even though the support is
      implemented, it is not fully enabled at this time. The logic to fallback
      to planner when the plan contains tables distributed with non-default
      non-legacy opclasses remains. Our intention is to support it fully in
      the future.
      
      How does this work?
      For hash joins, capture the opfamily of each hash joinable operator. Use
      that to create hash distribution spec requests for either side of the
      join.  Scan operators derive a distribution spec based on opfamily
      (corresponding to the opclass) of each distribution column.  If there is
      a mismatch between distribution spec requested/derived, add a Motion
      Redistribute node using the distribution function from the requested
      hash opfamily.
      
      The commit consists of several sub-sections:
      - Capture distr opfamilies in CMDRelation and related classes
      
        For each distribution column of the relation, track the opfamily of
        "opclass" used in the DISTRIBUTED BY clause. This information is then
        relayed to CTableDescriptor & CPhysicalScan.
      
        Also support this in other CMDRelation subclasses: CMDRelationCTAS
        (via CLogicalCTAS) & CMDRelationExternalGPDB.
      
      - Capture hash opfamily of CMDScalarOp using gpdb::GetCompatibleHashOpFamily()
        This is need to determined distribution spec requests from joins.
      
      - Track hash opfamilies of join predicates
      
        This commit extends the caching of join keys in Hash/Merge joins by
        also caching the corresponding hash opfamilies of the '=' operators
        used in those predicates.
      
      - Track opfamily in CDistributionSpecHashed.
      
        This commit also constructs CDistributionSpecHashed with opfamily
        information that was previously cached in CScalarGroup in the case of
        HashJoins.
        It also includes the compatibility checks that reject distributions
        specs with mismatched opfamilies in order to produce Redistribute
        motions.
      
      - Capture default distribution (hash) opfamily in CMDType
      - Handle legacy opfamilies in CMDScalarOp & CMDType
      - Handle opfamilies in HashExprList Expr->DXL translation
      
      ORCA-side notes:
      1. To ensure correctness, equivalent classes can only be determined over
         a specific opfamily. For example, the expression `a = b` implies a &
         b belong to an equiv classes only for the opfamily `=` belongs to.
         Otherwise expression `b |=| c` can be used to imply a & c belong to
         the same equiv class, which is incorrect, as the opfamily of `=` and
         `|=|` differ.
         For this commit, determine equiv classes only for default opfamilies.
         This will ensure correct behavior for majority of cases.
      2. This commit does *not* implement similar features for merge joins.
         That is left for future work.
      3. This commit introduces two traceflags:
         - EopttraceConsiderOpfamiliesForDistribution: If this is off,
           opfamilies is ignored and set to NULL. This mimics behavior before
           this PR. Ctest MDPs are run this way.
         - EopttraceUseLegacyOpfamilies: Set if ANY distribution col in the
           query uses a legacy opfamily/opclass. MDCache getters will then
           return legacy opfamilies instead of the default opfamilies for all
           queries.
      
      What new information is captured from GPDB?
      1. Opfamily of each distribution column in CMDRelation,
         CMDRelationCtasGPDB & CMDRelationExternalGPDB
      2. Compatible hash opfamily of each CMDScalarOp using
         gpdb::GetCompatibleHashOpFamily()
      3. Default distribution (hash) opfamily of every type.
         This maybe NULL for some types. Needed for certain operators (e.g
         HashAgg) that request distribution spec that cannot be inferred in
         any other way: cannot derive it, cannot get it from any scalar op
         etc. See GetDefaultDistributionOpfamilyForType()
      4. Legacy opfamilies for types & scalar operators.
         Needed for supporting tables distributed by legacy opclasses.
      
      Other GPDB side changes:
      
      1. HashExprList no longer carries the type of the expression (it is
         inferred from the expr instead). However, it now carries the hash
         opfamily to use when deriving the distribution hash function. To
         maintain compatibility with older versions, the opfamily is used only
         if EopttraceConsiderOpfamiliesForDistribution is set, otherwise,
         default hash distribution function of the type of the expr is used.
      2. Don't worry about left & right types in get_compatible_hash_opfamily()
      3. Consider COERCION_PATH_RELABELTYPE as binary coercible for ORCA.
      4. EopttraceUseLegacyOpfamilies is set if any table is distributed by a
         legacy opclass.
      5e04eb14
    • S
      Revert "Fallback when citext op non-citext join predicate is present" · 6921a50f
      Shreedhar Hardikar 提交于
      This reverts commit 3e45f064.
      6921a50f
  13. 09 4月, 2020 8 次提交
    • H
      Fix the nowait issue(#9881) · f67f6d29
      Hao Wu 提交于
      This is a backport of PR(https://github.com/greenplum-db/gpdb/pull/9726)
      for 6X_STABLE. NOWAIT only affects how SELCT locks rows as they are
      obtained from the table. They are used by the execution. So, the SELECT
      statement with locking clause will wait for the table lock if it
      can't hold the table lock.
      f67f6d29
    • H
      Avoid truncate error message in cdb_tidy_message (#9827) · 4146d9c8
      Hubert Zhang 提交于
      cdb_tidy_message is gpdb specific function. It will truncate the
      error message to just keep the first line and copy other lines to
      error details.
      For JDBC drivers, they follow the Postgres's error message and only
      keep error message in JDBC getWarnings function. It makes JDBC
      dirver cannot print the full error message of gpdb.
      It's better to follow postgres error format.
      
      (cherry picked from commit 0d0c33bf)
      4146d9c8
    • A
      Update orca test pipeline · 143f8b74
      Ashuka Xue 提交于
      Add a missing resource so that the ORCA dev pipeline can be run for both
      master and 6X.
      143f8b74
    • C
      Increase instance memory for gpexpand tests · dddd360b
      Chris Hajas 提交于
      The gpepxand tests were going OOM with ORCA in debug build, as the
      segments require more memory. Previously, ORCA was not being run in
      debug build so this was not an issue. We've bumped up the instance from
      n1-standard-2 to n1-standard-4, which doubles our memory from 7.5GB to
      15GB.
      dddd360b
    • A
      Harden resource group bypass test · 178a555a
      Ashuka Xue 提交于
      Previously, this test depended on a very specific amount of memory being
      "eaten" or consumed before it went OOM. ORCA using gpdb allocators
      vs the legacy allocation (which is used in debug builds for leak
      detection) changes the output. However, we have no way to detect or
      change the output file depending on whether we're in debug build or not.
      
      Now, instead of consuming the same amount of memory each time in the
      test, we consume a small amount and ensure we don't go OOM. Then we
      consume a larger amount and ensure we do go OOM. This hardens the test
      while keeping the intent.
      178a555a
    • C
      Stop cluster before changing system limits · 3a302414
      Chris Hajas 提交于
      During the behave tests, we change some of the system limits such as
      memory and overcommit configurations. Previously this was changed while
      the cluster was running, which caused the container to crash when
      setting some of these parameters with ORCA in debug build. Now, we stop
      the cluster, change the limits, and start the cluster.
      3a302414
    • H
      Add dependency info for C++ files · 13a31569
      Hans Zeller 提交于
      13a31569
    • H
      Merging Orca .editorconfig into gpdb file · 0ae1c1dc
      Hans Zeller 提交于
      0ae1c1dc