1. 28 10月, 2017 1 次提交
    • H
      When dispatching, send ActiveSnapshot along, not some random snapshot. · 4a95afc1
      Heikki Linnakangas 提交于
      If the caller specifies DF_WITH_SNAPSHOT, so that the command is dispatched
      to the segments with a snapshot, but it currently has no active snapshot in
      the QD itself, that seems like a mistake.
      
      In qdSerializeDtxContextInfo(), the comment talked about which snapshot to
      use when the transaction has already been aborted. I didn't quite
      understand that. I don't think the function is used to dispatch the "ABORT"
      statement itself, and we shouldn't be dispatching anything else in an
      already-aborted transaction.
      
      This makes it more clear which snapshot is dispatched along with the
      command. In theory, the latest or serializable snapshot can be different
      from the one being used when the command is dispatched, although I'm not
      sure if there are any such cases in practice.
      
      In the upcoming 8.4 merge, there are more changes coming up to snapshot
      management, which make it more difficult to get hold of the latest acquired
      snapshot in the transaction, so changing this now will ease the pain of
      merging that.
      
      I don't know why, but after making the change in qdSerializeDtxContextInfo,
      I started to get a lot of "Too many distributed transactions for snapshot
      (maxCount %d, count %d)" errors. Looking at the code, I don't understand
      how it ever worked. I don't see any no guarantee that the array in
      TempQDDtxContextInfo or TempDtxContextInfo was pre-allocated correctly.
      Or maybe it got allocated big enough to hold max_prepared_xacts, which
      was always large enough, but it seemed rather haphazard to me. So in
      the spirit of "if you don't understand it, rewrite it until you do", I
      changed the way the allocation of the inProgressXidArray array works.
      In statically allocated snapshots, i.e. SerializableSnapshot and
      LatestSnapshot, the array is malloc'd. In a snapshot copied with
      CopySnapshot(), it is points to a part of the palloc'd space for the
      snapshot. Nothing new so far, but I changed CopySnapshot() to set
      "maxCount" to -1 to indicate that it's not malloc'd. Then I modified
      DistributedSnapshot_Copy and DistributedSnapshot_Deserialize to not give up
      if the target array is not large enough, but enlarge it as needed. Finally,
      I made a little optimization in GetSnapshotData() when running in a QE, to
      move the copying of the distributed snapshot data to outside the section
      guarded by ProcArrayLock. ProcArrayLock can be heavily contended, so that's
      a nice little optimization anyway, but especially now that
      DistributedSnapshot_Copy() might need to realloc the array.
      4a95afc1
  2. 29 8月, 2017 2 次提交
  3. 25 8月, 2017 1 次提交
    • H
      Use ereport, rather than elog, for performance. · 01dff3ba
      Heikki Linnakangas 提交于
      ereport() has one subtle but important difference to elog: it doesn't
      evaluate its arguments, if the log level says that the message doesn't
      need to be printed. This makes a small but measurable difference in
      performance, if the arguments contain more complicated expressions, like
      function calls.
      
      While performance testing a workload with very short queries, I saw some
      CPU time being used in DtxContextToString. Those calls were coming from the
      arguments to elog() statements, and the result was always thrown away,
      because the log level was not high enough to actually log anything. Turn
      those elog()s into ereport()s, for speed.
      
      The problematic case here was a few elogs containing DtxContextToString
      calls, in hot codepaths, but I changed a few surrounding ones too, for
      consistency.
      
      Simplify the mock test, to not bother mocking elog(), while we're at it.
      The real elog/ereport work just fine in the mock environment.
      01dff3ba
  4. 09 8月, 2017 1 次提交
    • P
      Do not include gp-libpq-fe.h and gp-libpq-int.h in cdbconn.h · cf7cddf7
      Pengzhou Tang 提交于
      The whole cdb directory was shipped to end users and all header files
      that cdb*.h included are also need to be shipped to make checkinc.py
      pass. However, exposing gp_libpq_fe/*.h will confuse customer because
      they are almost the same as libpq/*, as Heikki's suggestion, we should
      keep gp_libpq_fe/* unchanged. So to make system work, we include
      gp-libpg-fe.h and gp-libpq-int.h directly in c files that need them
      cf7cddf7
  5. 02 8月, 2017 1 次提交
    • R
      Make memory spill in resource group take effect · 68babac4
      Richard Guo 提交于
      Resource group memory spill is similar to 'statement_mem' in
      resource queue, the difference is memory spill is calculated
      according to the memory quota of the resource group.
      
      The related GUCs, variables and functions shared by both resource
      queue and resource group are moved to the namespace resource manager.
      
      Also codes of resource queue relating to memory policy are refactored in this commit.
      Signed-off-by: NPengzhou Tang <ptang@pivotal.io>
      Signed-off-by: NNing Yu <nyu@pivotal.io>
      68babac4
  6. 06 7月, 2017 1 次提交
    • D
      Support an optional message in backend cancel/terminate (#2729) · fa6c2d43
      Daniel Gustafsson 提交于
      This adds the ability for the caller of pg_terminate_backend() or
      pg_cancel_backend() to include an optional message to the process
      which is being signalled. The message will be appended to the error
      message returned to the killed process. The new syntax is overloaded
      as:
      
          SELECT pg_terminate_backend(<pid> [, msg]);
          SELECT pg_cancel_backend(<pid> [, msg]);
      fa6c2d43
  7. 19 6月, 2017 1 次提交
  8. 07 6月, 2017 1 次提交
    • P
      restore TCP interconnect · 353a937d
      Pengzhou Tang 提交于
      This commit restore TCP interconnect and fix some hang issues.
      
      * restore TCP interconnect code
      * Add GUC called gp_interconnect_tcp_listener_backlog for tcp to control the backlog param of listen call
      * use memmove instead of memcpy because the memory areas do overlap.
      * call checkForCancelFromQD() for TCP interconnect if there are no data for a while, this can avoid QD from getting stuck.
      * revert cancelUnfinished related modification in 8d251945, otherwise some queries will get stuck
      * move and rename faultinjector "cursor_qe_reader_after_snapshot" to make test cases pass under TCP interconnect.
      353a937d
  9. 02 6月, 2017 1 次提交
    • X
      Remove subtransaction information from SharedLocalSnapshotSlot · b52ca70f
      Xin Zhang 提交于
      Originally, the reader kept copies of subtransaction information in
      two places.  First, it copied SharedLocalSnapshotSlot to share between
      writer and reader.  Second, reader kept another copy in subxbuf for
      better performance.  Due to lazy xid, subtransaction information can
      change in the writer asynchronously with respect to the reader.  This
      caused reader's subtransaction information out of date.
      
      This fix removes those copies of subtransaction information in the
      reader and adds a reference to the writer's PGPROC to
      SharedLocalSnapshotSlot.  Reader should refer to subtransaction
      information through writer's PGPROC and pg_subtrans.
      
      Also added is a lwlock per shared snapshot slot.  The lock protects
      shared snapshot information between a writer and readers belonging to
      the same session.
      
      Fixes github issues #2269 and #2284.
      Signed-off-by: NAsim R P <apraveen@pivotal.io>
      b52ca70f
  10. 01 6月, 2017 1 次提交
    • A
      Optimize DistributedSnapshot check and refactor to simplify. · 3c21b7d8
      Ashwin Agrawal 提交于
      Before this commit, snapshot stored information of distributed in-progress
      transactions populated during snapshot creation and its corresponding localXids
      found during tuple visibility check later (used as cache) by reverse mapping
      using single tightly coupled data structure DistributedSnapshotMapEntry. Storing
      the information this way possed couple of problems:
      
      1] Only one localXid can be cached for a distributedXid. For sub-transactions
      same distribXid can be associated with multiple localXid, but since can cache
      only one, for other local xids associated with distributedXid need to consult
      the distributed_log.
      
      2] While performing tuple visibility check, code must loop over full size of
      distributed in-progress array always first to check if cached localXid can be
      utilized to avoid reverse mapping.
      
      Now, decoupled the distributed in-progress with local xids cache separately. So,
      this allows us to store multiple xids per distributedXid. Also, allows to
      optimize scanning localXid only if tuple xid is relevant to it and also scanning
      size only equivalent to number of elements cached instead of size of distributed
      in-progress always even if nothing was cached.
      
      Along the way, refactored relevant code a bit as well to simplify further.
      3c21b7d8
  11. 04 5月, 2017 1 次提交
  12. 28 4月, 2017 1 次提交
    • A
      Correct calculation of xminAllDistributedSnapshots and set it on QE's. · d887fe0c
      Ashwin Agrawal 提交于
      For vacuum, page pruning and freezing to perform its job correctly on QE's, it
      needs to know globally what's the lowest dxid till any transaction can see in
      full cluster. Hence QD must calculate and send that info to QE. For this purpose
      using logic similar to one for calculating globalxmin by local snapshot. TMGXACT
      for global transactions serves similar to PROC and hence its leveraged to
      provide us lowest gxid for its snapshot. Further using its array, shmGxactArray,
      can easily find the lowest across all global snapshots and pass down to QE via
      snapshot.
      
      Adding unit test for createDtxSnapshot along with the change.
      d887fe0c
  13. 13 4月, 2017 1 次提交
    • A
      Fix dereference after null check in ProcArrayEndTransaction. · 7a9af586
      Ashwin Agrawal 提交于
      Coverity reported: Either the check against null is unnecessary, or there may be
      a null pointer dereference. In ProcArrayEndTransaction: Pointer is checked
      against null but then dereferenced anyway.
      
      While its not an issue and for commit case, pointer is never null, but simplify
      the code and stop using pointer itself here.
      7a9af586
  14. 07 4月, 2017 2 次提交
    • K
      Implement concurrency limit of resource group. · d0c6a352
      Kenan Yao 提交于
      Works include:
      * define structures used by resource group in shared memory;
      * insert/remove shared memory object when Create/Drop Resource Group;
      * clean up and restore when Create/Drop Resource Group fails;
      * implement concurrency slot acquire/release functionality;
      * sleep when concurrency slot is not available, and wake up others when
      releasing a concurrency slot if necessary;
      * handle signals in resource group properly;
      
      Signed-off-by Richard Guo <riguo@pivotal.io>
      Signed-off-by Gang Xiong <gxiong@pivotal.io>
      d0c6a352
    • K
      Since we added a GUC 'gp_resource_manager' to switch between resource queue · e630fb1f
      Kenan Yao 提交于
      and resource group when 'resource_scheduler' is on, we need to change the
      condition of the resource queue branches. Also, tidy up error messages related
      to resource manager under these different GUC settings.
      
      Signed-off-by Richard Guo <riguo@pivotal.io>
      Signed-off-by Gang Xiong <gxiong@pivotal.io>
      e630fb1f
  15. 01 4月, 2017 3 次提交
    • A
      Cleanup LocalDistribXactData related code. · 8c20bc94
      Ashwin Agrawal 提交于
      Commit fb86c90d "Simplify management of
      distributed transactions." cleanedup lot of code for LocalDistribXactData and
      introduced LocalDistribXactData in PROC for debugging purpose. But it's only
      correctly maintained for QE's, QD never populated LocalDistribXactData in
      MyProc. Instead TMGXACT also had LocalDistribXactData which was just set
      initially for QD but never updated later and confused more than serving the
      purpose. Hence removing LocalDistribXactData from TMGXACT, as it already has
      other fields which provide required information. Also, cleaned-up QD related
      states as even in PROC only QE uses LocalDistribXactData.
      8c20bc94
    • A
      Fully enable lazy XID allocation in GPDB. · 0932453d
      Ashwin Agrawal 提交于
      As part of 8.3 merge, upstream commit 295e6398
      "Implement lazy XID allocation" was merged. But transactionIds were still
      allocated in StartTransaction as code changes required to make it work for GPDB
      with distrbuted transaction was pending, thereby feature remained as
      disabled. Some progress was made by commit
      a54d84a3 "Avoid assigning an XID to
      DTX_CONTEXT_QE_AUTO_COMMIT_IMPLICIT queries." Now this commit addresses the
      pending work needed for handling deferred xid allocation correctly with
      distributed transactions and fully enables the feature.
      
      Important highlights of changes:
      
      1] Modify xlog write and xlog replay record for DISTRIBUTED_COMMIT. Even if
      transacion is read-only for master and no xid is allocated to it, it can still
      be distributed transaction and hence needs to persist itself in such a case. So,
      write xlog record even if no local xid is assigned but transaction is
      prepared. Similarly during xlog replay of the XLOG_XACT_DISTRIBUTED_COMMIT type,
      perform distributed commit recovery ignoring local commit. Which also means for
      this case don't commit to distrbuted log, as its only used to perform reverse
      map of localxid to distributed xid.
      
      2] Remove localXID from gxact, as its no more needed to be maintained and used.
      
      3] Refactor code for QE Reader StartTransaction. There used to be wait-loop with
      sleep checking to see if SharedLocalSnapshotSlot has distributed XID same as
      that of READER to assign reader some xid as that of writer, for SET type
      commands till READER actually performs GetSnapShotData(). Since now a) writer is
      not going to have valid xid till it performs some write, writers transactionId
      turns out InvalidTransaction always here and b) read operations like SET doesn't
      need xid, any more hence need for this wait is gone.
      
      4] Thow error if using distributed transaction without distributed xid. Earlier
      AssignTransactionId() was called for this case in StartTransaction() but such
      scenario doesn't exist hence convert it to ERROR.
      
      5] QD earlier during snapshot creation in createDtxSnapshot() was able to assign
      localXid in inProgressEntryArray corresponding to distribXid, as localXid was
      known by that time. That's no more the case and localXid mostly will get
      assigned after snapshot is taken. Hence now even for QD similar to QE's snapshot
      creation time localXid is not populated but later found in
      DistributedSnapshotWithLocalMapping_CommittedTest(). There is chance to optimize
      and try to match earlier behavior somewhat by populating gxact in
      AssignTransactionId() once locakXid is known but currently seems not so much
      worth it as QE's anyways have to perform the lookups.
      0932453d
    • A
      Optimize distributed xact commit check. · 692be1a1
      Ashwin Agrawal 提交于
      Leverage the fact that inProgressEntryArray is sorted based on distribXid while
      creating the snapshot in createDtxSnapshot. So, can break out fast in function
      DistributedSnapshotWithLocalMapping_CommittedTest().
      692be1a1
  16. 07 3月, 2017 3 次提交
    • A
      Checkpointer and BgWriter code closer to PG 9.2. · a453e7a3
      Ashwin Agrawal 提交于
      Rename checkpoint.c to checkpointer.c. And move the code from bgwriter.c to
      checkpointer.c and also renames most of corresponding data structures to refect
      the clear ownership and association. This commit brings it as close as possible
      to PostgreSQL 9.2.
      
      Reference to PostgreSQL related commits:
      commit 806a2aee
          Split work of bgwriter between 2 processes: bgwriter and checkpointer.
      commit bf405ba8
          Add new file for checkpointer.c
      commit 8f28789b
          Rename BgWriterShmem/Request to CheckpointerShmem/Request
      commit d843589e5ab361dd4738dab5c9016e704faf4153
          Fix management of pendingOpsTable in auxiliary processes.
      a453e7a3
    • A
      Fix checkpoint wait for CommitTransaction. · 787992e4
      Ashwin Agrawal 提交于
      `MyProc->inCommit` is to protect checkpoint running during inCommit
      transactions.
      
      However, `MyProc->lxid` has to be valid because `GetVirtualXIDsDelayingChkpt()`
      and `HaveVirtualXIDsDelayingChkpt()` require `VirtualTransactionIdIsValid()` in
      addition to `inCommit` to block the checkpoint process.
      
      In this fix, we defer clearing `inCommit` and `lxid` to `CommitTransaction()`.
      787992e4
    • A
      Use VXIDs instead of xid for checkpoint delay. · a02b9e99
      Ashwin Agrawal 提交于
      Originally checkpoint is checking for xid, however, xid is used to control the
      transaction visibility and it's crucial to clean this xid if process is done
      with commit and before release locks.
      
      However, checkpoint need to wait for the `AtExat_smgr()` to cleanup persistent
      table information, which happened after release locks, where `xid` is already
      cleaned.
      
      Hence, we use VXID, which doesn't have visibility impact.
      
      NOTE: Upstream PostgreSQL commit f21bb9cf for the
      similar fix.
      a02b9e99
  17. 10 2月, 2017 2 次提交
  18. 03 2月, 2017 1 次提交
    • A
      Revamp gp_fastsequence handling. · a34214e1
      Ashwin Agrawal 提交于
      Context:
      gp_fastsequence is used to generate and keep track of row numbers for AO and CO
      tables. Row numbers for AO/CO tables act as a component to form TID, stored in
      index tuples and used during index scans to lookup intended tuple. Hence this
      number must be monotonically incrementing value. Also should not rollback
      irrespective of insert/update transaction aborting for AO/CO table, as reusing
      row numbers even across aborted transactions would yield wrong results for index
      scans. Also, entries in gp_fastsequence only must exist for lifespan of the
      corresponding table.
      
      Change:
      Given those special needs, now reserved entries in gp_fastsequence are created
      as part of create table itself instead of deffering their creation to insert
      time. Insert within same transaction as create table is the only scenario needs
      coverage from these precreated entries, reserved entries above hence means entry
      for segfile 0 (used by CTAS or ALTER) and segfile 1 (used by insert within same
      transaction as create). Rest all entries continue to use frozen inserts to
      gp_fastsequence as they can only happen after create table transaction has
      committed.
      
      With that change in logic to leverage MVCC to handle cleanup of entries for
      gp_fastseqeunce, enables to get rid of special recovery and abort code
      performing frozen deletes. With that code gone fixes issues like:
      1] `REINDEX DATABASE` or `REINDEX TABLE pg_class` hang on segment nodes if
      encounters error after Prepare Transaction.
      
      2] Dangling gp_fastsequence in scenario, transaction created AO table, inserted
      tuples and aborts after prepare phase is complete. To cleanup gp_fastsequence,
      must open the relation and perform frozen heap delete to mark the entry as
      invisible. But if the backend performing the abort prepared is not connected to
      the same database, then delete operation cannot be done and leaves dangling
      entries.
      
      Output of helpful interaction with Heikki Linnakangas and Asim R P.
      See discussion on gpdb-dev, thread 'reindex database abort hang':
      https://groups.google.com/a/greenplum.org/forum/#!topic/gpdb-dev/ASml6lN0qRE
      a34214e1
  19. 25 1月, 2017 1 次提交
    • A
      Stop ignoring Lazy vacuum from RecentXmin calculation. · 7383c2b0
      Ashwin Agrawal 提交于
      As part of 8.3 merge via this upstream commit
      92c2ecc1, code to ignore lazy vacuum from
      calculating RecentXmin and RecentGlobalXmin was introduced.
      
      In GPDB as part of lazy vacuum, reindex is performed for bitmap indexes, which
      generates tuples in pg_class with lazy vacuum's transaction ID. Ignoring lazy
      vacuum from RecentXmin and RecentGlobalXmin during GetSnapshotData caused
      incorrect setting of hintbits to `HEAP_XMAX_INVALID` for tuple intended to be
      deletd by lazy vacuum and breaking HOT chain. This transaction visibility issue
      was encountered in CI many times with parallel schedule `bitmap_index, analyze`
      failing with error `could not find pg_class tuple for index` at commit time of
      lazy vacuum. Hence this commit stops tracking lazy vacuum in MyProc and
      performing any specific action related to same.
      7383c2b0
  20. 21 12月, 2016 1 次提交
    • A
      Update SharedLocalSnapshot correctly for subtransactions · 46d9521b
      Ashwin Agrawal 提交于
      QE reader leverages SharedLocalSnapshot to perform visibility checks. QE writer
      is responsible to keep the SharedLocalSnapshot up to date. Before this fix,
      SharedLocalSnapshot was only updated by writer while acquiring the snapshot. But
      if transaction id is assigned to subtransaction after it has taken the snapshot,
      it was not reflected. Due to this when QE reader called
      TransactionIdIsCurrentTransactionId, it may get sometimes false based on timings
      for subtransaction ids used by QE writer to insert/update tuples. Hence to fix
      the situation, SharedLocalSnapshot is now updated when assigning transaction id
      and deregistered if subtransaction aborts.
      
      Also, adding faultinjector to suspend cursor QE reader instead of guc/sleep used
      in past. Moving cursor tests from bugbuster to ICG and adding deterministic test
      to exercise the behavior.
      
      Fixes #1276, reported by @pengzhout
      46d9521b
  21. 26 8月, 2016 1 次提交
  22. 25 7月, 2016 1 次提交
    • K
      Fix the self-deadlock caused by reentrance of malloc/free when QD is in idle · c3fdfd74
      Kenan Yao 提交于
      state.
      
      There are two cases leading to this self-deadlock:
      (1) SIGALRM for IdleSessionGangTimeout comes when QD is in malloc function call
      of SSL code for example, and the handler HandleClientWaitTimeout would call
      function free to destroy Gangs, hence we are calling free inside malloc, which
      would produce a deadlock;
      (2) If a SIGUSR1 come when we are inside HandleClientWaitTimeout and calling
      function free, then we would be interrupted to process the Catchup event first,
      in which we would possibly call malloc, hence we are calling malloc inside free,
      which would cause a deadlock also.
      
      To fix this issue, for case 1, we only enable SIGALRM handling of IdleSessionGangTimeout
      exactly before recv call, which can protect malloc/free from being interrupted
      by this kind of SIGALRM; for case 2, we prevent reentrant signal handling.
      
      This fix is mainly borrowed from a patch of Tao Ma in Apache HAWK project.
      c3fdfd74
  23. 17 7月, 2016 1 次提交
  24. 16 7月, 2016 1 次提交
    • H
      Simplify management of distributed transactions. · fb86c90d
      Heikki Linnakangas 提交于
      We used to have a separate array of LocalDistributedXactData instances, and
      a reference in PGPROC to its associated LocalDistributedXact. That's
      unnecessarily complicated: we can store the LocalDistributedXact information
      directly in the PGPROC entry, and get rid fo the auxiliary array and the
      bookkeeping needed to manage that array.
      
      This doesn't affect the backend-private cache of committed Xids that also
      lives in cdblocaldistribxact.c.
      
      Now that the PGPROC->localDistributedXactData fields are never accessed
      by other backends, don't protect it with ProcArrayLock anymore. This makes
      the code simpler, and potentially improves performance too (ProcArrayLock
      can be very heavily contended on a busy system).
      fb86c90d
  25. 04 7月, 2016 1 次提交
    • D
      Use SIMPLE_FAULT_INJECTOR() macro where possible · 38741b45
      Daniel Gustafsson 提交于
      Callers to FaultInjector_InjectFaultIfSet() which don't pass neither
      databasename nor tablename and that use DDLNotSpecified can instead
      use the convenient macro SIMPLE_FAULT_INJECTOR() which cuts down on
      the boilerplate in the code. This commit does not bring any changes
      in functionality, merely readability.
      38741b45
  26. 28 6月, 2016 1 次提交
  27. 02 6月, 2016 1 次提交
    • H
      Remove checkpoint.h, and move the definitions in it to bgwriter.h. · a619808f
      Heikki Linnakangas 提交于
      Having a "checkpoint.h", corresponding to "checkpoint.c", makes perfect
      sense, but those function definitions are in bgwriter.h in PostgreSQL, and
      keeping the code as close to upstream as possible trumps the consistency of
      keeping definitions for "foo.c" in header file called "foo.h". Keeping
      things close to upstream makes merging easier.
      a619808f
  28. 19 5月, 2016 1 次提交
  29. 11 5月, 2016 1 次提交
    • H
      Avoid deadlock on catchup interrupt. · 698603da
      Heikki Linnakangas 提交于
      An earlier attempt at this checked AmIInSIGUSR1Handler() to see if we
      are currently processing a catchup event. But that's not good enough:
      we also process catchup interrupts outside the signal handler, in
      EnableCatchupInterrupt(). I saw lockups during "make installcheck-good"
      with a stack trace that shows a backend waiting for lock on a temporary
      relation, while trying to truncate it when committing the transaction
      opened for processing a catchup event.
      
      For reference, the commit message for the commit that introduced the
      AmIInSIGUSR1Handler check said:
      
          Recent parallel installcheck-good revealed we have a chance to process
          catchup interrupt while waiting for commit-prepare, and if the prepared
          transaction has created a temporary table with on commit option, the
          newly opened transaction for the sake of AcceptInvalidationMessages()
          cannot see and fails before the commit-prepare.  It's even not clear if
          we are safe to open and commit another transaction between prepare and
          commit-prepare, but for now just skip the oncommit operation as it
          doesn't have any effect anyway.
      698603da
  30. 10 5月, 2016 1 次提交
  31. 07 1月, 2016 1 次提交
    • A
      Remove gpverify code. · 8afc1dd1
      Ashwin Agrawal 提交于
      gpverify functionality was planned in past to deliver online
      verification of primary and mirror. It's completely dead code as never
      exposed this functionality and have no plans either. It eliminates
      creation of unused processes, extra directory, some shared memory and
      all.
      8afc1dd1
  32. 17 12月, 2015 1 次提交
  33. 14 12月, 2015 1 次提交
    • G
      Metadata Versioning feature for the ORCA Query Optimizer. · 6c31a3b4
      George Caragea 提交于
      Added a generation-based Metadata Versioning mechanism
      which will be used by ORCA to cache and invalidate catalog
      data in its Metadata Cache.
      Versioning is disabled by default at this point, until the
      Metadata Cache eviction policy is completed.
      6c31a3b4