1. 15 7月, 2020 2 次提交
    • H
      Cleanup idle reader gang after utility statements · d1ba4da5
      Hubert Zhang 提交于
      Reader gangs use local snapshot to access catalog, as a result, it will
      not synchronize with the sharedSnapshot from write gang which will
      lead to inconsistent visibility of catalog table on idle reader gang.
      Considering the case:
      
      select * from t, t t1; -- create a reader gang.
      begin;
      create role r1;
      set role r1;  -- set command will also dispatched to idle reader gang
      
      When set role command dispatched to idle reader gang, reader gang
      cannot see the new tuple t1 in catalog table pg_auth.
      To fix this issue, we should drop the idle reader gangs after each
      utility statement which may modify the catalog table.
      Reviewed-by: NZhenghua Lyu <zlv@pivotal.io>
      d1ba4da5
    • Z
      Correct plan of general & segmentGeneral path with volatiole functions. · d1f9b96b
      Zhenghua Lyu 提交于
      General and segmentGeneral locus imply that if the corresponding slice
      is executed in many different segments should provide the same result
      data set. Thus, in some cases, General and segmentGeneral can be
      treated like broadcast.
      
      But what if the segmentGeneral and general locus path contain volatile
      functions? volatile functions, by definition, do not guarantee results
      of different invokes. So for such cases, they lose the property and
      cannot be treated as *general. Previously, Greenplum planner
      does not handle these cases correctly. Limit general or segmentgeneral
      path also has such issue.
      
      The fix idea of this commit is: when we find the pattern (a general or
      segmentGeneral locus paths contain volatile functions), we create a
      motion path above it to turn its locus to singleQE and then create a
      projection path. Then the core job becomes how we choose the places to
      check:
      
        1. For a single base rel, we should only check its restriction, this is
           the at bottom of planner, this is at the function set_rel_pathlist
        2. When creating a join path, if the join locus is general or segmentGeneral,
           check its joinqual to see if it contains volatile functions
        3. When handling subquery, we will invoke set_subquery_pathlist function,
           at the end of this function, check the targetlist and havingQual
        4. When creating limit path, the check and change algorithm should also be used
        5. Correctly handle make_subplan
      
      OrderBy clause and Group Clause should be included in targetlist and handled
      by the above Step 3.
      
      Also this commit fixes DMLs on replicated table. Update & Delete Statement on
      a replicated table is special. These statements have to be dispatched to each
      segment to execute. So if they contain volatile functions in their targetList
      or where clause, we should reject such statements:
      
        1. For targetList, we check it at the function create_motion_path_for_upddel
        2. For where clause, they will be handled in the query planner and if we
           find the pattern and want to fix it, do another check if we are updating
           or deleting replicated table, if so reject the statement.
        3. Upsert case is handled in transform stage.
      d1f9b96b
  2. 14 7月, 2020 2 次提交
  3. 13 7月, 2020 2 次提交
    • Z
      Remove unused function pathnode_walk_node. · 7339a178
      Zhenghua Lyu 提交于
      Previously, `cdbpath_dedup_fixup` is the only function that
      will invoke `pathnode_walk_node`. And it was removed by the
      commit 9628a332.
      
      So in this commit we remove these unused functions.
      7339a178
    • (
      Fix flaky test for replication_keeps_crash. (#10423) · db60b003
      (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 info. Which lead the segment crash
      and starts recover.
      db60b003
  4. 10 7月, 2020 10 次提交
    • N
      ic-proxy: enable ic-proxy with --enable-ic-proxy · 81810a20
      Ning Yu 提交于
      We used to use the option --with-libuv to enable ic-proxy, it is not
      staightforward to understand the purpose of that option, though.  So we
      renamed it to --enable-ic-proxy, and the default setting is changed to
      "disable".
      
      Suggested by Kris Macoskey <kmacoskey@pivotal.io>
      81810a20
    • N
      ic-proxy: let backends connect to the proxy bgworker · 94c9d996
      Ning Yu 提交于
      Only in proxy mode, of course.  Currently the ic-proxy mode shares most
      of the backend logic with ic-tcp mode, so instead of copying the code we
      actually embed the ic-proxy specific logic in ic_tcp.c .
      94c9d996
    • N
      ic-proxy: launch as a bgworker · 5b60069c
      Ning Yu 提交于
      5b60069c
    • N
      ic-proxy: new value "proxy" in GUC gp_interconnect_type · 245ca266
      Ning Yu 提交于
      It is for the ic-proxy mode.
      245ca266
    • N
      ic-proxy: make gp_interconnect_proxy_addresses a GUC · 3140a44f
      Ning Yu 提交于
      3140a44f
    • N
      ic-proxy: implement the core logic · 6188fb1f
      Ning Yu 提交于
      The interconnect proxy mode, a.k.a. ic-proxy, is a new interconnect
      mode, all the backends communicate via a proxy bgworker, all the
      backends on the same segment share the same proxy bgworker, so every two
      segments only need one network connection between them, which reduces
      the network flows as well the ports.
      
      To enable the proxy mode we need to first configure the guc
      gp_interconnect_proxy_addresses, for example:
      
          gpconfig \
            -c gp_interconnect_proxy_addresses \
            -v "'1:-1:10.0.0.1:2000,2:0:10.0.0.2:2001,3:1:10.0.0.3:2002'" \
            --skipvalidation
      
      Then restart to take effect.
      6188fb1f
    • N
      Store dbid in CdbProcess · 8804bf39
      Ning Yu 提交于
      It is a preparation for the ic-proxy mode, we need this information to
      distinguish a primary segment with its mirror.
      8804bf39
    • A
      [Refactor] Pull out KHeap into CKHeap.h · 9e8f261d
      Ashuka Xue 提交于
      Pull out the implementation for binary heap into its own templated h
      file.
      9e8f261d
    • A
      Make histograms commutative when merging · 9b427611
      Ashuka Xue 提交于
      Prior to this commit, merging two histograms was not commutative.
      Meaning histogram1->Union(histogram2) could result in a row estimate of
      1500 rows, but histogram2->Union(histogram1) could result in a row
      estimate of 600 rows.
      
      Now, MakeBucketMerged has been renamed to SplitAndMergeBuckets. This
      function, which calculates the statistics for the merged bucket, now
      consistently return the same histogram buckets regardless of the order
      of input. This in turn, makes MakeUnionHistogramNormalize and
      MakeUnionAllHistogramNormalize commutative.
      
      Once we have successfully split the buckets and merged them as
      necessary, we may have generated up to 3X the number of buckets that
      were originally present. Thus we cap the number of buckets to be either
      the max size of the two incoming buckets, or, 100 buckets.
      
      CombineBuckets will then reduce the size of the histogram by combining
      consecutive buckets that have similar information. It does this by using
      a combination of two ratios: freq/ndv and freq/bucket_width. These two
      ratios were decided based off the following examples:
      
      Assuming that we calculate row counts for selections like the following:
      - For a predicate col = const: rows * freq / NDVs
      - For a predicate col < const: rows * (sum of full or fractional frequencies)
      
      Example 1 (rows = 100), freq/width, ndvs/width and ndvs/freq are all the same:
        ```
        Bucket 1: [0, 4)   freq .2  NDVs 2  width 4  freq/width = .05 ndv/width = .5 freq/ndv = .1
        Bucket 2: [4, 12)  freq .4  NDVs 4  width 8  freq/width = .05 ndv/width = .5 freq/ndv = .1
        Combined: [0, 12)  freq .6  NDVs 6  width 12
        ```
      
      This should give the same estimates for various predicates, with separate or combined buckets:
      ```
      pred          separate buckets         combined bucket   result
      -------       ---------------------    ---------------   -----------
      col = 3  ==>  100 * .2 / 2           = 100 * .6 / 6    = 10 rows
      col = 5  ==>  100 * .4 / 4           = 100 * .6 / 6    = 10 rows
      col < 6  ==>  100 * (.2 + .25 * .4)  = 100 * .5 * .6   = 30 rows
      ```
      
      Example 2 (rows = 100), freq and ndvs are the same, but width is different:
      ```
      Bucket 1: [0, 4)   freq .4  NDVs 4  width 4  freq/width = .1 ndv/width = 1 freq/ndv = .1
      Bucket 2: [4, 12)  freq .4  NDVs 4  width 8  freq/width = .05 ndv/width = .5 freq/ndv = .1
      Combined: [0, 12)  freq .8  NDVs 8  width 12
      ```
      
      This will give different estimates with the combined bucket, but only for non-equal preds:
      ```
      pred          separate buckets         combined bucket   results
      -------       ---------------------    ---------------   --------------
      col = 3  ==>  100 * .4 / 4           = 100 * .8 / 8    = 10 rows
      col = 5  ==>  100 * .4 / 4           = 100 * .8 / 8    = 10 rows
      col < 6  ==>  100 * (.4 + .25 * .4) != 100 * .5 * .8     50 vs. 40 rows
      ```
      
      Example 3 (rows = 100), now NDVs / freq is different:
      ```
      Bucket 1: [0, 4)   freq .2  NDVs 4  width 4  freq/width = .05 ndv/width = 1 freq/ndv = .05
      Bucket 2: [4, 12)  freq .4  NDVs 4  width 8  freq/width = .05 ndv/width = .5 freq/ndv = .1
      Combined: [0, 12)  freq .6  NDVs 8  width 12
      ```
      
      This will give different estimates with the combined bucket, but only for equal preds:
      ```
      pred          separate buckets         combined bucket   results
      -------       ---------------------    ---------------   ---------------
      col = 3  ==>  100 * .2 / 4          != 100 * .6 / 8      5 vs. 7.5 rows
      col = 5  ==>  100 * .4 / 4          != 100 * .8 / 8      10 vs. 7.5 rows
      col < 6  ==>  100 * (.2 + .25 * .4)  = 100 * .5 * .6   = 30 rows
      ```
      
      This commit also adds an attribute to the statsconfig for MaxStatsBuckets
      and changes the scaling method when creating singleton buckets.
      9b427611
    • A
      [Refactor] Update MakeStatsFilter, Rename CreateHistMashMapAfterMergingDisjPreds -> · c14fbb92
      Ashuka Xue 提交于
      MergeHistogramMapsforDisjPreds
      
      This commit refactors MakeStatsFilter to use
      MakeHistHashMapConjOrDisjFilter instead of individually calling
      MakeHistHashMapConj and MakeHistHashMapDisj.
      
      This commit also modifies MergeHistogramMapsForDisjPreds to avoid copy
      and creating unnecessary histogram buckets.
      c14fbb92
  5. 09 7月, 2020 3 次提交
  6. 07 7月, 2020 4 次提交
    • X
      Alter table add column on AOCS table inherits the default storage settings · 9a574915
      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.
      9a574915
    • H
      Disallow the replicated table inherit or to be inherited (#10344) · dc4b839e
      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>
      dc4b839e
    • C
      Move slack command trigger repo · 8e6a46f7
      Chris Hajas 提交于
      We've moved the repo that holds trigger commits to a private repo since
      there wasn't anything interesting there.
      8e6a46f7
    • A
      Fix vacuum on temporary AO table · 327abdb5
      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.
      327abdb5
  7. 06 7月, 2020 1 次提交
    • (
      Fix bitmap scan crash issue for AO/AOCS table. (#10407) · cb5d18d1
      (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>
      cb5d18d1
  8. 03 7月, 2020 1 次提交
    • P
      Force to update checkpoint timeline id in control data after promotion and... · e072dff7
      Paul Guo 提交于
      Force to update checkpoint timeline id in control data after promotion and before pg_rewind. (#10402)
      
      After promotion there is a checkpoint but not an immediate one, so the
      checkpoint timeline id in pg_control data might not be updated in time when
      running pg_rewind which depends on that to decide the necessity of incremental
      recovery.  Let's run checkpoint after promotion in tests so that later
      pg_rewind will not bypass the incremental recovery.
      
      This should be able to fix some long-existing annoying flaky pg_rewind tests.
      Reviewed-by: N(Jerome)Junfeng Yang <jeyang@pivotal.io>
      e072dff7
  9. 02 7月, 2020 1 次提交
    • H
      Fix Orca optimizer search stage couldn't measure elapsed time correctly · db25c3c8
      Haisheng Yuan 提交于
      Previously, CTimerUser didn't initialize timer, so the elapsed time provided by
      Orca was not meaningful, sometimes confusing.
      
      When traceflag T101012 is turned on, we can see the following trace message:
      
      [OPT]: Memo (stage 0): [20 groups, 0 duplicate groups, 44 group expressions, 4 activated xforms]
      [OPT]: stage 0 completed in 860087 msec,  plan with cost 1028.470667 was found
      [OPT]: <Begin Xforms - stage 0>
      ......
      [OPT]: <End Xforms - stage 0>
      [OPT]: Search terminated at stage 1/1
      [OPT]: Total Optimization Time: 67ms
      
      As shown above, the stage 0 elapsed timer is much greater than the total
      optimization time, which is obviously incorrect.
      db25c3c8
  10. 01 7月, 2020 3 次提交
    • (
      Let FTS mark mirror down if replication keeps crash. (#10327) · 252ba888
      (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.
      252ba888
    • Z
      Make QEs does not use the GUC gp_enable_global_deadlock_detector · 855f1548
      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: https://github.com/greenplum-db/gpdb/pull/9992#pullrequestreview-402364938.
      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 https://github.com/greenplum-db/gpdb/issues/10030  is gone after this commit.
      855f1548
    • H
      Fix memory accounting (#9739) · 048ca12b
      Hao Wu 提交于
      1. `localAllocated` is not set to 0 when resetting the memory context, which
      makes the statistics data incorrect. 
      2. Size is an unsigned integer, and `MEMORY_ACCOUNT_UPDATE_ALLOCATED`
      uses the opposite value for subtraction. Since they have the same bit width
      and the same bit results. We'd avoid using it in this way.
      3. Fix memory accounting when setting a new parent. The accounting
      parent must be one of the ancestors of the child. Re-setting the parent
      might break the rule, see below:
      
      If we set the new parent to the sibling or the ancestor of the old parent,
      all children's accounting parent may have to change, because
      the old accountingParent may not be the ancestor of the children's AllocSet.
      We must have a loop to find it. For example:
      
      root <- A <- B <- H <- C <- D
                     \
                       <-E
      root <- A <- B <- H
                     \
                       <-E <- C <- D
      
      We want to change the parent of C to E, and both C and D have the B
      to be the accountingParent. After we set the new parent of C to E,
      the accountingParent has to update. But if the new parent of C is B,
      it still satisfies the above rule. So, we don't have to update the accountingParent.
      
      The issue 1 is fixed in PR(https://github.com/greenplum-db/gpdb/pull/10106).
      Reviewed-by: NNing Yu <nyu@pivotal.io>
      048ca12b
  11. 30 6月, 2020 2 次提交
    • (
      Fix a cornor case which dump CaseTestExpr for IS NOT DISTINCT FROM. (#10348) · 5f58e117
      (Jerome)Junfeng Yang 提交于
      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 to become a RelabelType node and
      contains CaseTestExpr node in RelabelType->arg.
      
      So when dumping the view, it'll dump extra `CASE_TEST_EXPR` as below
      ```
      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.
      Reviewed-by: NPaul Guo <paulguo@gmail.com>
      5f58e117
    • P
      Fix assert failure in cdbcomponent_getCdbComponents() (#10355) · 06d7fe0a
      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>
      06d7fe0a
  12. 29 6月, 2020 4 次提交
  13. 24 6月, 2020 5 次提交
    • H
      Clean up the internal functions to merge partition stats a bit. · b22544fb
      Heikki Linnakangas 提交于
      aggregate_leaf_partition_MCVs() and aggregate_leaf_partition_histograms()
      functions did catalog lookups to count how many leaf partitions a root
      table has. Why? Because the caller passes two arrays as inputs, with an
      entry for each partition, and the functions need to know how large the
      input arrays are. That's overly complicated, of course: the caller can
      simply pass the size of the arrays as an argument. That's much more robust
      too, I think the current code would crash and burn if the partition
      hierarchy was modified concurrently. I'm not sure if that's a live bug, or
      if we're holding locks that prevent that, but let's keep things simple in
      any case.
      
      This removes the last callers of rel_get_leaf_children_relids() function,
      so remove that altogether.
      Reviewed-by: NAbhijit Subramanya <asubramanya@pivotal.io>
      Reviewed-by: NHans Zeller <hzeller@vmware.com>
      b22544fb
    • H
      Fix DNS lookup issue for pure IPv6 addresses in URLs (#10331) · 40443808
      Huiliang.liu 提交于
      *Problem:
      getDnsCachedAddress() always returns empty string if the hostname only
      has IPv6 addresses
      
      *Root cause:
      Array hostinfo[] is redeclared in code section for IPv6, so the real
      lookup result never returns.
      
      *How to reproduce:
      You can reproduce this issue even you don't have pure IPv6 environment
      1. Assign a IPv6 address for your default net device
         sudo ip -6 addr add 2401::1234/128 dev ${netdev}
      2. Add the address to /etc/hosts with a dummy hostname, for example
         2401::1234/128	ipv6host
      3. Create external table using the dummy hostname
         create external table test_ext_ipv6(a int) location('gpfdist://ipv6host:8080/data_file') format 'csv';
      4. Launch gpfdist
      5. Run test SQL
         select * from test_ext_ipv6
         Then an error will be reported with an empty address in url:
         ERROR:  connection with gpfdist failed for "gpfdist://ipv6host:8080/data_file", effective url: "http://:8080/data_file"
      Co-authored-by: NPeifeng Qiu <pqiu@pivotal.io>
      40443808
    • T
      Remove lockfile from mainUtils · 8190ed40
      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>
      8190ed40
    • J
      Check whether the directory exists when deleting the tablespace (#10305) · b1b99c43
      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.
      b1b99c43
    • H
      Only apply transformGroupedWindows() with ORCA. (#10306) · e52dd032
      Heikki Linnakangas 提交于
      * Only apply transformGroupedWindows() with ORCA.
      
      The Postgres planner doesn't need it. Move the code to do it, so that it's
      only used before passing a tree to ORCA. This doesn't change anything with
      ORCA, but with the Postgres planner, it has some benefits:
      
      * Some cases before this patch do not give correct results and now they run
        correctly (e.g. case `regress/olap_window_seq`)
      * Fixes github issue #10143.
      
      * Make transformGroupedWindows walk the entire tree
      
      The transformGroupedWindows function now recursively transforms any
      Query node in the tree that has both window functions and groupby or
      aggregates.
      
      Also fixed a pre-existing bug where we put a subquery in the target
      list of such a Query node into the upper query, Q'. This meant that
      any outer references to the scope of Q' no longer had the correct
      varattno. The fix is to place the subquery into the target list of
      the lower query, Q'' instead, which has the same range table as the
      original query Q. Therefore, the varattnos to outer references to the
      scope of Q (now Q'') don't need to be updated. Note that varlevelsup to
      scopes above Q still need to be adjusted, since we inserted a new
      scope Q'. (See comments in code for explanations of Q, Q', Q'').
      Co-authored-by: NHeikki Linnakangas <hlinnakangas@pivotal.io>
      Co-authored-by: NHans Zeller <hzeller@vmware.com>
      Co-authored-by: NAbhijit Subramanya <asubramanya@pivotal.io>
      e52dd032