1. 15 7月, 2020 6 次提交
  2. 13 7月, 2020 2 次提交
    • D
      Docs - remove HCI warning · e53fbea1
      David Yozie 提交于
      e53fbea1
    • (
      Fix flaky test for replication_keeps_crash. (#10427) · 1350c1b6
      (Jerome)Junfeng Yang 提交于
      Remove the set `gp_fts_probe_retries to 1` which may cause FTS probe failed.
      This was first added to reduce the test time, but set a lower retry
      value may cause the test failed to probe FTS update segment
      configuration. Since reduce the `gp_fts_replication_attempt_count` also
      save the test time, so skip alter ``gp_fts_probe_retries`.
      
      Also find an assertion may not match when mark mirror down happens before
      walsender exit, which will free the replication status before walsender
      exit and try to record disconnect time and failure count. Which lead the
      segment crash and starts recover.
      1350c1b6
  3. 10 7月, 2020 1 次提交
    • X
      Alter table add column on AOCS table inherits the default storage settings · 4834e2ee
      xiong-gang 提交于
      When alter table add a column to AOCS table, the storage setting (compresstype,
      compresslevel and blocksize) of the new column can be specified in the ENCODING
      clause; it inherits the setting from the table if ENCODING is not specified; it
      will use the value from GUC 'gp_default_storage_options' when the table dosen't
      have the compression configuration.
      4834e2ee
  4. 09 7月, 2020 4 次提交
    • H
      Disallow the replicated table inherit or to be inherited(#10428) · 6c842d0e
      Hao Wu 提交于
      Currently, replicated tables are not allowed to inherit a parent
      table. But ALTER TABLE .. INHERIT can pass around the restriction.
      
      On the other hand, a replicated table is allowed to be inherited
      by a hash distributed table. It makes things much complicated.
      When the parent table is declared as a replicated table inherited by
      a hash distributed table, its data on the parent is replicated
      but the data on the child is hash distributed. When running
      `select * from parent;`, the generated plan is:
      ```
      gpadmin=# explain select * from parent;
                                       QUERY PLAN
      -----------------------------------------------------------------------------
       Gather Motion 3:1  (slice1; segments: 3)  (cost=0.00..4.42 rows=14 width=6)
         ->  Append  (cost=0.00..4.14 rows=5 width=6)
               ->  Result  (cost=0.00..1.20 rows=4 width=7)
                     One-Time Filter: (gp_execution_segment() = 1)
                     ->  Seq Scan on parent  (cost=0.00..1.10 rows=4 width=7)
               ->  Seq Scan on child  (cost=0.00..3.04 rows=2 width=4)
       Optimizer: Postgres query optimizer
      (7 rows)
      ```
      It's not particularly useful for the parent table to be replicated.
      So, we disallow the replicated table to be inherited.
      Reported-by: NHeikki Linnakangas <hlinnakangas@pivotal.io>
      Reviewed-by: NHubert Zhang <hzhang@pivotal.io>
      (cherry picked from commit dc4b839e)
      6c842d0e
    • X
      Fix flaky test gp_replica_check · d6e4e7ef
      xiong-gang 提交于
      When there is a big lag between primary and mirror replay, gp_replica_check
      will fail if the checkpoint is not replayed in about 60 seconds. Extend the
      timeout to 600 seconds to reduce the chance of flaky.
      d6e4e7ef
    • T
      Remove lockfile from mainUtils · 988f961e
      Tyler Ramer 提交于
      [Lockfile](https://pypi.org/project/lockfile/) has not been maintained
      since around 2015. Further, the functionality it provided seems poor - a
      review of the code indicated that it used the presence of the PID file
      itself as the lock - in Unix, using a file's existence followed by a
      creation is not atomic, so a lock could be prone to race conditions.
      
      The lockfile package also did not clean up after itself - a process
      which was destroyed unexpectedly would not clear the created locks, so
      some faulty logic was added to mainUtils.py, which checked to see if a
      process with the same PID as the lockfile's creator was running. This
      is obviously failure prone, as a new process might be assigned the same
      PID as the old lockfile's owner, without actually being the same process.
      
      (Of note, the SIG_DFL argument to os.kill() is not a signal at all, but
      rather of type signal.handler. It appears that the python cast this
      handler to the int 0, which, according to man 2 kill, leads to no signal
      being sent, but existance and permission checks are still performed. So
      it is a happy accident that this code worked at all)
      
      This commit removes lockfile from the codebase entirely.
      
      It also adds a "PIDLockFile" class which provides an atomic-guarenteed
      lock via the mkdir and rmdir commands on Unix - thus, it is not safely
      portable to Windows, but this should not be an issue as only Unix-based
      utilities use the "simple_main()" function.
      
      PIDLockFile provides API compatible classes to replace most of the
      functionality from lockfile.PidLockFile, but does remove any timeout
      logic as it was not used in any meaningful sense - a hard-coded timeout
      of 1 second was used, but an immediate result of if the lock is held is
      sufficient.
      
      PIDLockFile also includes appropriate __enter__, __exit__, and __del__
      attributes, so that, should we extend this class in the future, with
      syntax is functional, and __del__ calls release, so a process reaped
      unexpectedly should still clean its own locks as part of the garbage
      collection process.
      Authored-by: NTyler Ramer <tramer@pivotal.io>
      
      Do not remove PYLIB_SRC_EXT during make clean/distclean
      
      Commit 8190ed40 removed lockfile from
      mainUtils, but did not remove a reference to its source directory in the
      make clean/distclean target. As a result, because LOCKFILE_DIR is no
      longer defined, the make clean/distclean target removes the
      PYLIB_SRC_EXT directory.
      988f961e
    • C
      Fix colrefs getting mangled when merging equivalent classes in Orca · 046f970a
      Chris Hajas 提交于
      Previously, the PdrgpcrsAddEquivClass function would modify the input
      colref set. This does not appear intentional, as this same reference may
      be accessed in other places. This caused Orca to fall back to planner in
      some cases during translation with "Attribute number 0 not found in
      project list".
      Co-authored-by: Nmubo.fy <mubo.fy@alibaba-inc.com>
      Co-authored-by: NChris Hajas <chajas@pivotal.io>
      Co-authored-by: NHans Zeller <hzeller@vmware.com>
      (cherry picked from commit 84c027afdff13f1d91447a5a88809f6e85399a1b)
      046f970a
  5. 08 7月, 2020 6 次提交
    • X
      gpcheckcat: fix gpcheckcat vpinfo issue · 6f6be4d9
      xiong-gang 提交于
      The entry in aocsseg table might be compacted and waiting for drop, so we
      should use 'state' to filter the unused entry.
      6f6be4d9
    • X
      gpcheckcat: add the check of vpinfo consistency · b76db711
      xiong-gang 提交于
      column 'vpinfo' in pg_aoseg.pg_aocsseg_xxx record the 'eof' of each attribute
      in the AOCS table. Add a new check 'aoseg_table' in gpcheckcat, it checks the
      number of attributes in 'vpinfo' is the same as the number of attributes in
      'pg_attribute'. This check is performed in parallel and independently on each
      segment, and it checks aoseg table and pg_attribute in different transaction,
      so it should be run 'offline' to avoid false alarm.
      b76db711
    • (
      Fix bitmap scan crash issue for AO/AOCS table. (#10422) · 0799b771
      (Jerome)Junfeng Yang 提交于
      When ExecReScanBitmapHeapScan get executed, bitmap state (tbmiterator
      and tbmres) gets freed in freeBitmapState. So the tbmres is NULL, and we
      need to reinit bitmap state to start scan from the beginning and reset AO/AOCS
      bitmap pages' flags(baos_gotpage, baos_lossy, baos_cindex and baos_ntuples).
      
      Especially when ExecReScan happens on the bitmap append only scan and
      not all the matched tuples in bitmap are consumed, for example, Bitmap
      Heap Scan as inner plan of the Nest Loop Semi Join. If tbmres not get init,
      and not read all tuples in last bitmap, BitmapAppendOnlyNext will assume the
      current bitmap page still has data to return. but bitmap state already freed.
      
      From the code, for Nest Loop Semi Join, when a match find, a new outer slot is
      requested, and then `ExecReScanBitmapHeapScan` get called, `node->tbmres` and
      `node->tbmiterator` set to NULL. `node->baos_gotpage` still keeps true.
      When execute `BitmapAppendOnlyNext`, it skip create new `node->tbmres`.
      And jump to access `tbmres->recheck`.
      Reviewed-by: NJinbao Chen <jinchen@pivotal.io>
      Reviewed-by: NAsim R P <pasim@vmware.com>
      (cherry picked from commit cb5d18d1)
      0799b771
    • (
      gpMgmt: make some utilities search_path-safe (#10394) · 483b0fc5
      (Jerome)Junfeng Yang 提交于
      Make some utilities search path safe, so it'll not calling any external
      functions that has the same name with our built-in functions.
      
      This fix does not guarantee to fix CVE-2018-1058.
      
      Backport from 070d6221.
      
      Co-authored-by: Jamie McAtamney jmcatamney@pivotal.io
      Co-authored-by: Jacob Champion pchampion@pivotal.io
      Co-authored-by: Shoaib Lari slari@pivotal.io
      483b0fc5
    • A
      Fix vacuum on temporary AO table · 0a6da752
      Ashwin Agrawal 提交于
      The path constructed in OpenAOSegmentFile() didn't take into account
      "t_" semantic of filename. Ideally, the correct filename is passed to
      function, so no need to construct the same.
      
      Would be better if can move MakeAOSegmentFileName() inside
      OpenAOSegmentFile(), as all callers call it except
      truncate_ao_perFile(), which doesn't fit that model.
      
      (cherry picked from commit 327abdb5)
      0a6da752
    • L
      docs - gphdfs2pxf migration pxf supports avro compression (#10415) · 21bcabf8
      Lisa Owen 提交于
      * docs - gphdfs2pxf migration pxf supports avro compression
      
      * missing plural
      21bcabf8
  6. 07 7月, 2020 1 次提交
  7. 06 7月, 2020 1 次提交
  8. 03 7月, 2020 3 次提交
  9. 02 7月, 2020 3 次提交
    • (
      Let FTS mark mirror down if replication keeps crash.(#10411) · 4c8cfe99
      (Jerome)Junfeng Yang 提交于
      For GPDB FTS, if the primary, mirror replication keeps crash
      continuously and attempt to crate replication connection too many times,
      FTS should mark the mirror down. Otherwise, it may block other
      processes.
      If the WAL starts streaming, clear the attempt count to 0. This is because the blocked
      transaction can only be released once the WAL in streaming state.
      
      The solution for this is:
      
      1. Use ` FTSReplicationStatus` which under `gp_replication.c`  to track current primary-mirror
      replication status. This includes:
          - A continuous failure counter. The counter gets reset once the replication
          starts streaming, or replication restarted.
          - A record of the last disconnect timestamp which is refactored from
          `WalSnd` slot.
          The reason for moving this is: When FTS probe happens, the `WalSnd`
          slot may already get freed. And `WalSnd` slot is designed reusable.
          It's hacky to read value from a freed slot in shared memory.
      
      2. When handling each probe query, `GetMirrorStatus` will check the current
      mirror status and the failure count from walsender's application ` FTSReplicationStatus`.
      If the count exceeds the limit, the retry test will ignore the last replication
      disconnect time since it gets refreshed when new walsender starts. (Since
      in the current case, the walsender keeps restart.)
      
      3. On FTS bgworker. If mirror down and retry set to false, mark the mirror
      down.
      
      A `gp_fts_replication_attempt_count` GUC is added. When the replication failure count
      exceed this GUC, ignore the last replication disconnect time when checking for mirror
      probe retry.
      
      The life cycle of a ` FTSReplicationStatus`:
      1. It gets created when first enable replication during the replication
      start phase. Each replication's sender should have a unique
      `application_name`, which also used to specify the replication priority
      in multi-mirror env. So ` FTSReplicationStatus` uses the `application_name` mark
      itself.
      
      2. The ` FTSReplicationStatus` for replication will exist until FTS detects
      failure and stop the replication between primary and mirror. Then
      ` FTSReplicationStatus` for that `application_name` will be dropped.
      
      Now the `FTSReplicationStatus` is used only for GPDB primary-mirror replication.
      
      (cherry picked from commit 252ba888)
      
      Reviewed-by: Ashwin Agrawal aagrawal@pivotal.io
      4c8cfe99
    • J
      Check whether the directory exists when deleting the tablespace (#10388) · 270a775b
      Jinbao Chen 提交于
      If the directory of tablespace does not exist, we should got a
      error on commit transaction. But error on commit transaction will
      cause a panic. So the directory of tablespace should be checked
      so that we can avoid panic.
      270a775b
    • Z
      Make QEs does not use the GUC gp_enable_global_deadlock_detector. · fde4af75
      Zhenghua Lyu 提交于
      Previously, there are some code executed in QEs that will check
      the value of the GUC gp_enable_global_deadlock_detector. The
      historical reason is that:
        Before we have GDD, UPDATE|DELETE operations cannot be
        concurrently executed, and we do not meet the EvalPlanQual
        issue and concurrent split update issue. After we have GDD,
        we meet such issues and add code to resolve them, and we
        add the code `if (gp_enable_global_deadlock_detector)` at
        the very first. It is just like habitual thinking.
      
      In fact, we do not rely on it in QEs and I tried to remove this
      in the context: #9992 (review).
      I tried to add an assert there but could not handle utility mode
      as Heikki's comments. To continue that idea, we can just remove
      the check of gp_enable_global_deadlock_detector. This brings two
      benefits:
        1. Some users only change this GUC on master. By removing the usage
           in QEs, it is safe now to make the GUC master only.
        2. We can bring back the skills to only restart master node's postmaster
           to enable GDD, this save a lot of time in pipeline. This commit
           also do this for the isolation2 test cases: lockmodes and gdd/.
      
      The Github issue #10030  is gone after this commit.
      
      Also, this commit backports lockmodes test from master branch and
      refactor concurrent_update test cases.
      fde4af75
  10. 01 7月, 2020 1 次提交
    • F
      README: Fix documentation to install Greenplum with ORCA in Ubuntu (#10253) · 335b99ad
      Francisco Guerrero 提交于
      While installing ORCA in Ubuntu 18.04 I ran into some issues while
      trying to install gp-xerces. During the build, Greenplum was trying to
      link against the installed xerces `libxerces-c-dev`, which is installed
      by running the `./README.ubuntu.bash` script. By default, Ubuntu 18.04
      will install xerces 3.2, which is incompatible with the version used by
      Greenplum 6.
      335b99ad
  11. 30 6月, 2020 7 次提交
    • J
      Fix binary_swap failure. · c17af285
      Junfeng(Jerome) Yang 提交于
      The pg_get_viewdef() function was fixed to properly show views define.
      Should remove the view which not supported in old version before the
      binary swap test.
      c17af285
    • H
      Skip foreign table when gpexpand · 5b508da5
      Hubert Zhang 提交于
      For non partition table, we will skip gpexpand external tables,
      but for partition table, when one of its child parition is a
      external table, we error out when gpexpand. This is not a correct
      behavior.
      Since data of foreign table is located outside gpdb, skip these
      tables when gpexpand is enough.
      
      Co-Authored-by: Ning Yu nyu@pivotal.io
      
      (cherry picked from commit b9691eba)
      5b508da5
    • (
      Fix CASE WHEN IS NOT DISTINCT FROM clause incorrect dump. (#10365) · 70eee180
      (Jerome)Junfeng Yang 提交于
      The clause 'CASE WHEN (arg1) IS NOT DISTINCT FROM (arg2)' dump will miss
      the arg1. For example:
      ```
      CREATE OR REPLACE VIEW xxxtest AS
      SELECT
          CASE
          WHEN 'I will disappear' IS NOT DISTINCT FROM ''::text
          THEN 'A'::text
          ELSE 'B'::text
          END AS t;
      ```
      The dump will lose 'I will disappear'.
      
      ```
      SELECT
          CASE
          WHEN IS NOT DISTINCT FROM ''::text
          THEN 'A'::text
          ELSE 'B'::text
          END AS t;
      ```
      
      For below example:
      ```
      CREATE TABLE mytable2 (
          key character varying(20) NOT NULL,
          key_value character varying(50)
      ) DISTRIBUTED BY (key);
      
      CREATE VIEW aaa AS
      SELECT
          CASE mytable2.key_value
              WHEN IS NOT DISTINCT FROM 'NULL'::text THEN 'now'::text::date
              ELSE to_date(mytable2.key_value::text, 'YYYYMM'::text)
              END AS t
          FROM mytable2;
      
      ```
      
      mytable2.key_value will cast to type date. For clause `(ARG1) IS NOT
      DISTINCT FROM (ARG2)`, this leads ARG1 become a RelabelType node and
      contains CaseTestExpr node in RelabelType->arg.
      
      So when dumping the view, it'll mark dump as
      
      ```
      select pg_get_viewdef('notdisview3',false);
                                     pg_get_viewdef
      -----------------------------------------------------------------------------
        SELECT                                                                    +
               CASE mytable2.key_value                                            +
                   WHEN (CASE_TEST_EXPR) IS NOT DISTINCT FROM 'NULL'::text THEN ('now'::text)::date+
                   ELSE to_date((mytable2.key_value)::text, 'YYYYMM'::text)       +
               END AS t                                                           +
          FROM mytable2;
      (1 row)
      ```
      
      I dig into commit a453004e, if left-hand argument for `IS NOT DISTINCT
      FROM` contains any `CaseTestExpr` node, the left-hand arg should be omitted.
      `CaseTestExpr` is a placeholder for CASE expression.
      70eee180
    • N
      resgroup: fix the cpu value of the per host status view · 9c81451f
      Ning Yu 提交于
      Resource group we does not distinguish the per segment cpu usage, the
      cpu usage reported by a segment is actually the total cpu usage of all
      the segments on the host.  This is by design, not a bug.  However, in
      the gp_toolkit.gp_resgroup_status_per_host view it reports the cpu usage
      as the sum of all the segments on the same host, so the reported per
      host cpu usage is actually N times of the actual usage, where N is the
      count of the segments on that host.
      
      Fixed by reporting the avg() instead of the sum().
      
      Tests are not provided as the resgroup/resgroup_views did not verify cpu
      usages since the beginning, because the cpu usage is unstable on
      pipelines.  However, I have verified manually.
      Reviewed-by: NHubert Zhang <hzhang@pivotal.io>
      (cherry picked from commit e0d78729)
      9c81451f
    • P
      Fix assert failure in cdbcomponent_getCdbComponents() (#10355) · 8b6e19ab
      Paul Guo 提交于
      It could be called in utility mode, however we should avoid calling into
      FtsNotifyProber() for such case. Ideally we could do that if we access the
      master node and postgres is no in single mode, but it seems that we do not that
      need.
      
      Here is the stack of the issue I encountered.
      
      0  0x0000003397e32625 in raise () from /lib64/libc.so.6
      1  0x0000003397e33e05 in abort () from /lib64/libc.so.6
      2  0x0000000000b39844 in ExceptionalCondition (conditionName=0xeebac0 "!(Gp_role == GP_ROLE_DISPATCH)", errorType=0xeebaaf "FailedAssertion",
          fileName=0xeeba67 "cdbfts.c", lineNumber=101) at assert.c:66
      3  0x0000000000bffb1e in FtsNotifyProber () at cdbfts.c:101
      4  0x0000000000c389a4 in cdbcomponent_getCdbComponents () at cdbutil.c:714
      5  0x0000000000c26e3a in gp_pgdatabase__ (fcinfo=0x7ffd7b009c10) at cdbpgdatabase.c:74
      6  0x000000000076dbd6 in ExecMakeTableFunctionResult (funcexpr=0x3230fc0, econtext=0x3230988, argContext=0x3232b68, expectedDesc=0x3231df8, randomAccess=0 '\000',
      	    operatorMemKB=32768) at execQual.c:2275
      ......
      18 0x00000000009afeb2 in exec_simple_query (query_string=0x316ce20 "select * from gp_pgdatabase") at postgres.c:1778
      Reviewed-by: NHubert Zhang <hzhang@pivotal.io>
      
      Cherry-picked from 06d7fe0a
      8b6e19ab
    • D
      Doc updates to cover packaging changes/additions for versions 7/6 (#10364) · 4631aedb
      David Yozie 提交于
      * Doc updates to cover packaging changes/additions for versions 7/6
      
      * Removing nav link to non-default steps
      
      * Add missing chown examples; add note about symbolic link update in upgrade procedure
      
      * Typo fix
      4631aedb
    • D
      Docs - update PXF version info · ab59c8dd
      David Yozie 提交于
      ab59c8dd
  12. 29 6月, 2020 4 次提交
    • A
      Skip external tables during analyze · a990ebde
      Ashwin Agrawal 提交于
      Analyze error'd with "ERROR: unsupported table type" if external table
      is part of partition hierarchy. Also, that blocks analyzing database
      wide as can't analyze such partitions. This commit adds check to skip
      external tables during acquire_inherited_sample_rows().
      
      Fixes #9506 github issue.
      Co-authored-by: NAlexandra Wang <lewang@pivotal.io>
      a990ebde
    • (
      Fix tupdesc dangling pointer segfault in HashAgg (#10384) · ef010af2
      (Jerome)Junfeng Yang 提交于
      This problem manifests itself with HashAgg on the top of
      DynamicIndexScan node and can cause a segmentation fault.
      
      1. A HashAgg node initializes a tuple descriptor for its hash
      slot using a reference from input tuples (coming from
      DynamicIndexScan through a Sequence node).
      2. At the end of every partition index scan in DynamicIndexScan
      we unlink and free unused memory chunks and reset partition's
      memory context. It causes a total destruction of all objects in
      the context including partition index tuple descriptor used in a
      HashAgg node.
      As a result we get a dangling pointer in HashAgg on switching to
      a new index partition during DynamicIndexScan that can cause a
      segfault.
      
      This backports from commit: 41ce55bf.
      Co-authored-by: NDenis Smirnov <sd@arenadata.io>
      ef010af2
    • N
      gpexpand: cleanup new segments in parallel · 328f215f
      Ning Yu 提交于
      When cleaning up the master-only files on the new segments we used to do
      the job one by one, when there are tens or hundreds of segments it can
      be very slow.
      
      Now we cleanup in parallel.
      
      (cherry picked from commit 857763ae)
      Reviewed-by: NZhenghua Lyu <zlv@pivotal.io>
      328f215f
    • E
      For Python testing artifacts, introduce combination of Concourse cache and pip --cache-dir. · 8ee3e05f
      Ed Espino 提交于
      For the Python testing artifacts used by the CLI tools, utilize the
      Concourse cached directories feature to create and use a pip cache dir
      shared between task runs.
      
      Be aware, the cache is scoped to the worker the task is run on. We do
      not get a cache hit when subsequent builds run on different workers.
      
      * The environment variable PIP_CACHE_DIR is used to store the cache
      directory.
      
      * Add "--retries 10" to Behave test dependency pip install commands.
      8ee3e05f
  13. 26 6月, 2020 1 次提交
    • D
      Fix dbid inconsistency on spread mirroring · 7111ec3a
      Denis Smirnov 提交于
      Mirror registration passes through several steps at the moment:
      1. CREATE_QE_ARRAY (QE_MIRROR_ARRAY is ordered by content)
      2. ARRAY_REORDER (QE_MIRROR_ARRAY is ordered by port)
      3. CREATE_ARRAY_SORTED_ON_CONTENT_ID (form QE_MIRROR_ARRAY_SORTED_ON_CONTENT_ID
         on a base of QE_MIRROR_ARRAY)
      4. REGISTER_MIRRORS (walk through QE_MIRROR_ARRAY, register mirrors with
         pg_catalog.gp_add_segment_mirror on master's gp_segment_configuration
         and update QE_MIRROR_ARRAY with returned dbids)
      5. CREATE_SEGMENT (walk through QE_MIRROR_ARRAY_SORTED_ON_CONTENT_ID with old
         dbids and create mirrors on segment hosts with pg_basebackup)
      The problem is in a step 4 - we update the wrong array (QE_MIRROR_ARRAY instead
      of QE_MIRROR_ARRAY_SORTED_ON_CONTENT_ID). Because of that we get inconsistency
      between mirror dbids on gp_segment_configuration and internal.auto.conf files.
      This can cause inoperable cluster state in some situations when we promote a
      failed primary from a mirror with wrong dbids (FTS can't solve this issue).
      
      Also fixed column indexes in array used for segment arrray ordering.
      It was not done after commit https://github.com/greenplum-db/gpdb/commit/03c7d557720c5a78af1e2574ac385d10a0797f5e
      which prepend array with new hostname column.
      Co-authored-by: NVasiliy Ivanov <ivi@arenadata.io>
      (cherry picked from commit 8d008792)
      7111ec3a