1. 28 10月, 2020 8 次提交
  2. 27 10月, 2020 3 次提交
    • X
      postgres_fdw: disable UPDATE/DELETE on foreign Greenplum servers · e1fed42a
      Xiaoran Wang 提交于
      Greenplum only supports INSERT, because UPDATE/DELETE requires the
      hidden column gp_segment_id and the other "ModifyTable mixes distributed
      and entry-only tables" issue.
      e1fed42a
    • C
      Remove Orca assertions when merging buckets · 44621b6f
      Chris Hajas 提交于
      These assertions started getting tripped in the previous commit when
      adding tests, but aren't related to the Epsilon change. Rather, we're
      calculating the frequency of a singleton bucket using two different
      methods which causes this assertion to break down. The first method
      (calculating the upper_third) assumes the singleton has 1 NDV and that there is an even distribution
      across the NDVs. The second (in GetOverlapPercentage) calculates a
      "resolution" that is based on Epsilon and assumes the bucket contains
      some small Epsilon frequency. It results in the overlap percentage being
      too high, instead it too should likely be based on the NDV.
      
      In practice, this won't have much impact unless the NDV is very small.
      Additionally, the conditional logic is based on the bounds, not
      frequency. However, it would be good to align in the future so our
      statistics calculations are simpler to understand and predictable.
      
      For now, we'll remove the assertions and add a TODO. Once we align the
      methods, we should add these assertions back.
      44621b6f
    • C
      Fix stats bucket logic for Double values in UNION queries in Orca · 45e49e17
      Chris Hajas 提交于
      When merging statistics buckets for UNION and UNION ALL queries
      involving a column that maps to Double (eg: floats, numeric, time
      related types), we could end up in an infinite loop. This occurred if
      the bucket boundaries that we compared were within a very small value,
      defined in Orca as Epsilon. While we considered that two values were
      equal if they were within Epsilon, we didn't when computing whether
      datum1 < datum2. Therefore we'd get into a situation where a datum
      could be both equal to and less than another datum, which the logic
      wasn't able to handle.
      
      The fix is to make sure we have a hard boundary of when we consider a
      datum less than another datum by including the epsilon logic in all
      datum comparisons. Now, 2 datums are equal if they are within epsilon,
      but datum1 is less than datum 2 only if datum1 < datum2 - epsilon.
      
      Also add some tests since we didn't have any tests for types that mapped
      to Double.
      45e49e17
  3. 26 10月, 2020 1 次提交
  4. 23 10月, 2020 4 次提交
    • D
      Fix CLOSE_WAIT leaks when Gang recycling · 30b52372
      dh-cloud 提交于
      Postgresql libpq document:
      
      > Note that when PQconnectStart or PQconnectStartParams returns a
      > non-null pointer, you must call PQfinish when you are finished
      > with it, in order to dispose of the structure and any associated
      > memory blocks. **This must be done even if the connection attempt
      > fails or is abandoned**.
      
      However, cdbconn_disconnect() function did not call PQfinish when
      CONNECTION_BAD, it can cause socket leaks (CLOSE_WAIT state).
      30b52372
    • P
      Fix an orphaned prepared transaction case due to race between checkpointer and... · 29aa03b9
      Paul Guo 提交于
      Fix an orphaned prepared transaction case due to race between checkpointer and COMMIT PREPARE xlog recording
      
      On Greenplum, checkpoint would collect prepared transactions which are actually
      committed. If the COMMIT PREPARE xlog is before checkpoint.redo, after the
      segment reboot, there would always be an orphaned (actually committed) prepared
      transaction in memory. That happens when we collect the prepared transaction in
      checkpointer before gxact->valid is reset and after the COMMIT PREPARE xlog is
      recorded, see code in FinishPreparedTransaction().
      
      That could lead to various issues. e.g. dtx recovery would keep trying to abort
      that and then cause panic on the segment with message like "cannot abort
      transaction 3285003, it was already committed (twophase.c:2205)".
      
      Fixing this by adding a new variable committed in gxact to specifiy whether the
      global transaction is committed or not. If being committed we surely do not
      need to log the gxact in checkpointer xlog. We could also fix this by delaying
      checkpointer later after gxact->valid resetting in
      FinishPreparedTransaction()), but RecordTransactionCommitPrepared() ->
      SyncRepWaitForLSN() might be time-consuming or block for some time somehow
      (locking, network lag, etc), thus it could block checkpointer for too long time
      - that is surely not good. Also it seems that we could fix that by moving
      "gxact->valid = false" ahead of delayChkpt resetting, but that is kind of ugly
      also, also that is a bit risky for a stable release.
      
      There were two solutions that were discussed previously. One is to use locking
      mechanism, but that hurts OLTP performance; Another is to remove the false
      positive cases in RecoverPreparedTransactions(), but it is possible that
      related clog has been removed by subsequent vacuum operations so it is not
      reliable also.
      Co-authored-by: NHao Wu <gfphoenix78@gmail.com>
      Reviewed-by: NAsim R P <pasim@vmware.com>
      Reviewed-by: NAshwin Agrawal <aashwin@vmware.com>
      29aa03b9
    • P
      Save errno across LWLockRelease() calls · 98b1c3f9
      Peter Eisentraut 提交于
      Fixup for "Drop slot's LWLock before returning from SaveSlotToPath()"
      Reported-by: NMichael Paquier <michael@paquier.xyz>
      (cherry picked from commit 72b2b9c52e3a86ae414fc07acf6db3de0776fc13)
      98b1c3f9
    • P
      Drop slot's LWLock before returning from SaveSlotToPath() · 32893eaf
      Peter Eisentraut 提交于
      When SaveSlotToPath() is called with elevel=LOG, the early exits didn't
      release the slot's io_in_progress_lock.
      
      This could result in a walsender being stuck on the lock forever.  A
      possible way to get into this situation is if the offending code paths
      are triggered in a low disk space situation.
      
      Author: Pavan Deolasee <pavan.deolasee@2ndquadrant.com>
      Reported-by: NCraig Ringer <craig@2ndquadrant.com>
      Discussion: https://www.postgresql.org/message-id/flat/56a138c5-de61-f553-7e8f-6789296de785%402ndquadrant.com
      (cherry picked from commit ce28a43ffa89b49584e75d6bb9f8ae03a8e13151)
      32893eaf
  5. 22 10月, 2020 3 次提交
  6. 21 10月, 2020 2 次提交
  7. 20 10月, 2020 2 次提交
    • H
      Use palloc() instead of palloc0() in buildGpQueryString() (#10996) · f3b01b4d
      Hao Wu 提交于
      The buildGpQueryString() function should use palloc() instead of palloc0()
      to allocate the buffer for holding the serialized query string, thus
      avoiding the unnecessary cost of setting all bytes in buffer to zero.
      Reviewed-by: NHeikki Linnakangas <hlinnaka@iki.fi>
      Reviewed-by: NAshwin Agrawal <aashwin@vmware.com>
      (cherry picked from commit 073be714)
      Co-authored-by: NFang Zheng <f.zheng@alibaba-inc.com>
      f3b01b4d
    • H
      Disable changing distribution keys implicitly when creating unique index (#10969) · 2505f1f5
      Hao Wu 提交于
      In previous GPDB version, the distribution keys may be changed implicitly
      when creating a unique index on a hash-distributed empty table.
      ```SQL
      create table foo(a int, b int) distributed by(a);
      create unique index on foo(b);
      -- now, foo is hash distributed by b, not by a
      ```
      It might be useful(maybe) to avoid changing the distribution keys. However,
      on the other side, it's crazy if the user doesn't notice the NOTICE message
      like, "NOTICE:  updating distribution policy to match new UNIQUE index".
      
      What's worse, this behavior could bring data inconsistency. See,
      ```SQL
      create table foo(a int, b int) distributed by(a);
      insert into foo select i,i from generate_series(1,5)i;
      
      create table foopart (i int4, j int4) distributed by (i) partition by
              range (i) (start (1) end (3) every (1));
      create unique index on foopart_1_prt_1 (j);
      insert into foopart values(1,2),(2,1);
      ```
      The data inconsistency is
      ```
      gpadmin=# select gp_segment_id, * from foopart_1_prt_1;
       gp_segment_id | i | j
       ---------------+---+---
                   1 | 1 | 2
       (1 row)
      
      gpadmin=# select * from foo f, foopart_1_prt_1 p where f.a = p.j;
       a | b | i | j
       ---+---+---+---
       (0 rows)
      ```
      
      Implicitly changing the distribution keys is not very useful, but harmful.
      This PR disables changing the distribution keys when creating a unique index.
      Reviewed-by: NHubert Zhang <hzhang@pivotal.io>
      2505f1f5
  8. 19 10月, 2020 1 次提交
    • P
      Fix flaky test regress/partition · fbc0a7be
      Paul Guo 提交于
      Test partition1 creates table hhh_r1 also and that test runs with test
      partition in parallel with schedule file greenplum_schedule, and this
      could cause below test failure. Fixing this by renaming the table name.
      
      @@ -523,9 +523,7 @@
       partition aa start (date '2007-01-01') end (date '2008-01-01')
             every (interval '0 days')
       );
      -ERROR:  EVERY parameter too small
      -LINE 5:       every (interval '0 days')
      -                     ^
      +ERROR:  relation "hhh_r1" already exists
       create table foo_p (i int) distributed by(i)
       partition by range(i)
       (start (1) end (20) every(0));
      Reviewed-by: NAsim R P <pasim@vmware.com>
      (cherry picked from commit 624f460c)
      fbc0a7be
  9. 16 10月, 2020 1 次提交
  10. 15 10月, 2020 5 次提交
    • S
      Increment ExternalScan::scancounter across queries in ORCA · de9b0e26
      Shreedhar Hardikar 提交于
      gpfdist uses the global xid & timestamp to distinguish whether each
      connection belongs to the same external scan or not.
      
      ORCA generates a unique scan number for each ExternalScan within the
      same plan, but not accross plans. So, within a transaction, we may issue
      multiple external scans that do not get differentiated properly,
      producing different results.
      
      This commit patches that by using a different scan number accross plans,
      just like what planner does. Ideally gpfdist should also take into
      account the command-id of the query to prevent this problem for other
      cases such as prepared statements.
      de9b0e26
    • X
      Fix some postgres_fdw issues (#10748) · 972697e7
      Xiaoran Wang 提交于
      * Fix and test postgres_fdw test
      Modify postgres_fdw test cases to run it in GP.
      1) Set up postgres_fdw progress by postgres_init.sh, postgres_fdw.sql
      are run in gpdb, and it connects to Postgres database by postgres_fdw.
      We don't use the lookback server as the remote Postgres database.
      2) Skip some EXPLAIN cases, as GP's plans sometimes are different from postgres
      and the EXPLAIN cases are not essential.
      3) "UPDATA" trigger can not work correctly in GP, just skip those related cases.
      
      * INSERT RETURNING issue
      For the partition table, the tupleDesc between child partition
      table and parent partition table may be different, because
      the parent partition table firstly drops a column, then adds a new
      partition. The new child partition doesn't contain the dropped
      column and shouldn't care about it when inserting data into the
      new partition. But when processing the RETURNING, we should use
      the parent slot as there is something in the planner to process the
      dropped column.
      
      For the normal table, the values and nulls in parentslot (which are
      inserted into the dest table) are not changed in ExecInsert
      function, but for foreign table, there may be triggers on the
      remote table, it can change the values, so we need to update the
      parentslot values for the foreign table after inserting the data
      into the remote dest table. Otherwise, the RETURNING expressions
      will work on incorrect data.
      
      * postgres_fdw test compiling Postgres from src
      
      A Postgres server is needed by the postgres_fdw test.
      So we download Postgres src and install it.
      Co-authored-by: Nfanfuxiaoran <wxiaoran@vmvare.com>
      972697e7
    • M
      Explicitly handle on QE side zero value parameter incoming from QD · 8a60b662
      Maksim Milyutin 提交于
      The master node dispatches external and internal (gathered from plan
      tree) parameters along with query plan. Internal parameters might
      include not initialized zero values, e.g., used for storing results of
      not yet evaluated initPlans. Those parameters are trasmitted as zero
      values that cases segfault on segments under deserialization of complex
      type value.
      
      The current fix intercepts on QE side handling of zero value parameter
      before further deserialization process.
      
      See GitHub PR #10773 for more information.
      8a60b662
    • H
      Fix DNS lookup issue for pure IPv6 addresses in URLs (#10331) · eb80e07a
      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>
      (cherry picked from commit 40443808)
      eb80e07a
    • H
      Fix DNS lookups of IPv6 addresses in URLs of web external tables. · 2965efd2
      Heikki Linnakangas 提交于
      The code to look up an IP address based on a domain name checked
      '!hostname', but 'hostname' is a local array, so the condition was never
      true. When compiling with -Waddress, GCC gave a warning about it:
      
      cdbutil.c: In function ‘getDnsCachedAddress’:
      cdbutil.c:1239:23: warning: the address of ‘hostinfo’ will always evaluate as ‘true’ [-Waddress]
         if (((!use_cache && !hostinfo) || (use_cache && e == NULL))
                             ^
      
      You don't see this normally, because we set -Wno-address in configure.in,
      but it sure looks wrong.
      
      I haven't been able to find a test case where this would cause a
      user-visible failure. I think you'd need a domain name that resolves to
      something that's not an IPv4 address, and use that domain in an external
      table's URL.
      
      (cherry picked from commit 48a96209)
      2965efd2
  11. 14 10月, 2020 2 次提交
  12. 13 10月, 2020 2 次提交
    • L
      LIKE - clause edit to highlight partitions limitation (#10935) · 96670d1b
      Lena Hunter 提交于
      * clarifying pg_upgrade note
      
      * LIKE clause edits
      
      * typo
      96670d1b
    • F
      Reduce overhead due to dtm and snapshot debug logging · 5780a31a
      Fang Zheng 提交于
      elog(DTM_DEBUG5, ...) (or elog((Debug_print_full_dtm ? LOG : DEBUG5),
      ...)) and ereport(DTM_DEBUG5, ...)  (or ereport((Debug_print_full_dtm
      ? LOG : DEBUG5), ...)) are replaced with elogif(Debug_print_full_dtm,
      LOG, ...) majority of places (on in some cases
      ereportif(Debug_print_full_dtm, LOG)).  Similar changes are made for
      logging statements related to Debug_print_snapshot_dtm as well.
      
      This way, in a production setup, each logging statement only pays the
      cost of checking the value of Debug_print_full_dtm, instead of first
      evaluating (Debug_print_full_dtm ? LOG : DEBUG5) and then checking log
      level. Besides, the changes also avoid unnecessarily evaluating
      parameters passed to elog() (e.g., call to DtxContextToString()).
      
      The changes slightly affect the behavior for debugging. When
      log_min_messages is set to DEBUG5, the original code that uses
      elog(DTM_DEBUG5, ...) emits the log message at DEBUG5 level if
      Debug_print_full_dtm is set to false. The new code that uses
      elogif(Debug_print_full_dtm, LOG, ...) does not emit the log message
      unless Debug_print_full_dtm is set to true. This means that a
      developer needs to set log_min_messages to DEBUG5 AND
      Debug_print_full_dtm/Debug_print_snapshot_dtm to true to get the full
      set of debug messages.
      
      With the changes, DTM_DEBUG3 and DTM_DEBUG5 macros are no longer used,
      hence deleting as part of this commit.
      Reviewed-by: NAshwin Agrawal <aashwin@vmware.com>
      5780a31a
  13. 12 10月, 2020 6 次提交
    • N
      ic-proxy: support hostname as proxy addresses · bd59bc5c
      Ning Yu 提交于
      The GUC gp_interconnect_proxy_addresses is used to set the listener
      addresses and ports of all the proxy bgworkers, only IP addresses were
      supported previously, which is inconvenient to use.
      
      Now we add the support for hostnames too, the IP addresses are also
      supported.
      
      Note that if a hostname is bound to a different IP at runtime, we must
      reload the setting with the "gpstop -u" command.
      Reviewed-by: NHubert Zhang <hzhang@pivotal.io>
      (cherry picked from commit 2a1794bc)
      bd59bc5c
    • P
      Fix flaky test isolation2:terminate_in_gang_creation (#10800) · 2d773773
      Paul Guo 提交于
      The test depends on fault create_gang_in_progress. Previously the fault might
      be hit by dtx recovery when creating a new gang. Let's skip checking in dtx
      recovery to avoid the test flakiness.
      
      The test flakiness looks like this:
      
       10&: SELECT * FROM foo a JOIN foo b USING (c2);  <waiting ...>
       +FAILED:  Forked command is not blocking; got output: c2 | c1 | c1
       +----+----+----
       +(0 rows)
      
       We could easily reproduce this by adding some sleep before the query.
      Reviewed-by: NAsim R P <pasim@vmware.com>
      
      Cherry-picked from 20e80056
      2d773773
    • P
      Periodically check and abort orphaned transactions post dtx recovery. · eea513bb
      Paul Guo 提交于
      Orphaned transactions are annoying in real production environments since they
      waste disks, hold related locks, etc. They could be sometimes generated in a
      non-buggy scenario (e.g. master reset/failover). Let's periodically check and
      abort them after dtm starts in dtx recovery. (i.e. we do not quit dtx recovery
      now).
      
      Added two gucs also.
      
      - gp_dtx_recovery_interval: interval to do checking.
      
      - gp_dtx_recovery_prepared_period: Fetch prepared transactions older than this
        value for checking.
      
      Currently the code checks orphaned prepared transactions only but I'd like to
      avoid 2pc abort retry panic using this code change in another commit.
      Reviewed-by: NAsim R P <pasim@vmware.com>
      
      (cherry picked from commit 5b73cefc)
      with some needed modification.
      eea513bb
    • W
      Skip FK check when do relation truncate · 924c4d3a
      Weinan WANG 提交于
      GPDB does not support FK, but keep FK grammar in DDL, since it
      reduce DB migration manual workload from others.
      Hence, we do not need FK to check for truncate command, rid of it.
      924c4d3a
    • A
      Partially fix CTAS pg_resgroup_get_status() (#10901) · 80eaf9c8
      Adam Lee 提交于
      The query `INSERT INTO .. SELECT .. FROM pg_resgroup_get_status()` will
      execute the UDF on a QE started on the master node, it's not right to
      check `if (Gp_role == GP_ROLE_DISPATCH)`.
      
      It's not a complete fix since 6X could not have a catalog change, a more
      complex query might start the QE on a segment that has no such stats.
      Better than nothing.
      80eaf9c8
    • Y
      Add function ExecStoreTuple for compatibility · 176e0462
      Yandong Yao 提交于
      Unlike PostgreSQL 9/10/11, Greenplum has no API ExecStoreTuple,
      thus some external libraries such as redis_fdw will not compile
      against Greenplum 6. Greenplum 7 has no such issue, as PostgreSQL 12
      changed ExecStoreTuple to ExecStoreHeapTuple like Greenplum.
      176e0462