You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@age.apache.org by jo...@apache.org on 2022/04/25 18:35:39 UTC

[incubator-age] branch master updated: Support for Agtype Containment Ops and GIN Indices

This is an automated email from the ASF dual-hosted git repository.

joshinnis pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-age.git


The following commit(s) were added to refs/heads/master by this push:
     new 6279c10  Support for Agtype Containment Ops and GIN Indices
6279c10 is described below

commit 6279c1043d9095a5c06593d5dc3193f3875ffc14
Author: Josh Innis <Jo...@gmail.com>
AuthorDate: Mon Apr 25 11:13:16 2022 -0700

    Support for Agtype Containment Ops and GIN Indices
    
    Add support for @>, <@, ?, ?|, ?& operators postgres operators.
    
    Alter property contraint checks in MATCH clause to use the @>  operator.
    
    Add support for GIN Indices
---
 Makefile                                |   1 +
 age--1.0.0.sql                          | 158 ++++++++-
 regress/expected/agtype.out             |  98 +++++-
 regress/expected/index.out              |  88 +++--
 regress/sql/agtype.sql                  |  27 +-
 regress/sql/index.sql                   |  50 +--
 src/backend/executor/cypher_set.c       |  18 +-
 src/backend/parser/cypher_clause.c      |  82 +++--
 src/backend/utils/adt/agtype.c          |  22 +-
 src/backend/utils/adt/agtype_gin.c      | 553 ++++++++++++++++++++++++++++++++
 src/backend/utils/adt/agtype_ops.c      | 160 +++++++++
 src/backend/utils/load/ag_load_edges.c  |  20 +-
 src/backend/utils/load/ag_load_labels.c |  27 +-
 13 files changed, 1147 insertions(+), 157 deletions(-)

diff --git a/Makefile b/Makefile
index ad92302..0745f07 100644
--- a/Makefile
+++ b/Makefile
@@ -51,6 +51,7 @@ OBJS = src/backend/age.o \
        src/backend/utils/adt/age_graphid_ds.o \
        src/backend/utils/adt/agtype.o \
        src/backend/utils/adt/agtype_ext.o \
+       src/backend/utils/adt/agtype_gin.o \
        src/backend/utils/adt/agtype_ops.o \
        src/backend/utils/adt/agtype_parser.o \
        src/backend/utils/adt/agtype_util.o \
diff --git a/age--1.0.0.sql b/age--1.0.0.sql
index bf4ac9a..712dfcc 100644
--- a/age--1.0.0.sql
+++ b/age--1.0.0.sql
@@ -2833,6 +2833,157 @@ CREATE OPERATOR CLASS agtype_ops_hash
   OPERATOR 1 =,
   FUNCTION 1 ag_catalog.agtype_hash_cmp(agtype);
 
+--
+-- Contains operators @> <@
+--
+CREATE FUNCTION ag_catalog.agtype_contains(agtype, agtype)
+RETURNS boolean
+LANGUAGE c
+STABLE
+RETURNS NULL ON NULL INPUT
+PARALLEL SAFE
+AS 'MODULE_PATHNAME';
+
+CREATE OPERATOR @> (
+  LEFTARG = agtype,
+  RIGHTARG = agtype,
+  FUNCTION = ag_catalog.agtype_contains,
+  COMMUTATOR = '<@',
+  RESTRICT = contsel,
+  JOIN = contjoinsel
+);
+
+CREATE FUNCTION ag_catalog.agtype_contained_by(agtype, agtype)
+RETURNS boolean
+LANGUAGE c
+STABLE
+RETURNS NULL ON NULL INPUT
+PARALLEL SAFE
+AS 'MODULE_PATHNAME';
+
+CREATE OPERATOR <@ (
+  LEFTARG = agtype,
+  RIGHTARG = agtype,
+  FUNCTION = ag_catalog.agtype_contained_by,
+  COMMUTATOR = '@>',
+  RESTRICT = contsel,
+  JOIN = contjoinsel
+);
+
+--
+-- Key Existence Operators ? ?| ?&
+--
+CREATE FUNCTION ag_catalog.agtype_exists(agtype, text)
+RETURNS boolean
+LANGUAGE c
+IMMUTABLE
+RETURNS NULL ON NULL INPUT
+PARALLEL SAFE
+AS 'MODULE_PATHNAME';
+
+CREATE OPERATOR ? (
+  LEFTARG = agtype,
+  RIGHTARG = text,
+  FUNCTION = ag_catalog.agtype_exists,
+  COMMUTATOR = '?',
+  RESTRICT = contsel,
+  JOIN = contjoinsel
+);
+
+CREATE FUNCTION ag_catalog.agtype_exists_any(agtype, text[])
+RETURNS boolean
+LANGUAGE c
+IMMUTABLE
+RETURNS NULL ON NULL INPUT
+PARALLEL SAFE
+AS 'MODULE_PATHNAME';
+
+CREATE OPERATOR ?| (
+  LEFTARG = agtype,
+  RIGHTARG = text[],
+  FUNCTION = ag_catalog.agtype_exists_any,
+  RESTRICT = contsel,
+  JOIN = contjoinsel
+);
+
+CREATE FUNCTION ag_catalog.agtype_exists_all(agtype, text[])
+RETURNS boolean
+LANGUAGE c
+IMMUTABLE
+RETURNS NULL ON NULL INPUT
+PARALLEL SAFE
+AS 'MODULE_PATHNAME';
+
+CREATE OPERATOR ?& (
+  LEFTARG = agtype,
+  RIGHTARG = text[],
+  FUNCTION = ag_catalog.agtype_exists_all,
+  RESTRICT = contsel,
+  JOIN = contjoinsel
+);
+
+--
+-- agtype GIN support
+--
+CREATE FUNCTION ag_catalog.gin_compare_agtype(text, text)
+RETURNS int
+AS 'MODULE_PATHNAME'
+LANGUAGE C
+IMMUTABLE
+STRICT
+PARALLEL SAFE;
+
+CREATE FUNCTION gin_extract_agtype(agtype, internal)
+RETURNS internal
+AS 'MODULE_PATHNAME'
+LANGUAGE C
+IMMUTABLE
+STRICT
+PARALLEL SAFE;
+
+CREATE FUNCTION ag_catalog.gin_extract_agtype_query(agtype, internal, int2,
+                                                    internal, internal)
+RETURNS internal
+AS 'MODULE_PATHNAME'
+LANGUAGE C
+IMMUTABLE
+STRICT
+PARALLEL SAFE;
+
+CREATE FUNCTION ag_catalog.gin_consistent_agtype(internal, int2, agtype, int4,
+                                                 internal, internal)
+RETURNS bool
+AS 'MODULE_PATHNAME'
+LANGUAGE C
+IMMUTABLE
+STRICT
+PARALLEL SAFE;
+
+CREATE FUNCTION ag_catalog.gin_triconsistent_agtype(internal, int2, agtype, int4,
+                                                    internal, internal, internal)
+RETURNS bool
+AS 'MODULE_PATHNAME'
+LANGUAGE C
+IMMUTABLE
+STRICT
+PARALLEL SAFE;
+
+CREATE OPERATOR CLASS ag_catalog.gin_agtype_ops
+DEFAULT FOR TYPE agtype USING gin AS
+  OPERATOR 7 @>,
+  OPERATOR 9 ?(agtype, text),
+  OPERATOR 10 ?|(agtype, text[]),
+  OPERATOR 11 ?&(agtype, text[]),
+  FUNCTION 1 ag_catalog.gin_compare_agtype(text,text),
+  FUNCTION 2 ag_catalog.gin_extract_agtype(agtype, internal),
+  FUNCTION 3 ag_catalog.gin_extract_agtype_query(agtype, internal, int2,
+                                                 internal, internal),
+  FUNCTION 4 ag_catalog.gin_consistent_agtype(internal, int2, agtype, int4,
+                                              internal, internal),
+  FUNCTION 6 ag_catalog.gin_triconsistent_agtype(internal, int2, agtype, int4,
+                                                 internal, internal, internal),
+STORAGE text;
+
 --
 -- graph id conversion function
 --
@@ -3325,13 +3476,6 @@ RETURNS NULL ON NULL INPUT
 PARALLEL SAFE
 AS 'MODULE_PATHNAME';
 
-CREATE FUNCTION ag_catalog._property_constraint_check(agtype, agtype)
-RETURNS boolean
-LANGUAGE c
-STABLE
-PARALLEL SAFE
-AS 'MODULE_PATHNAME';
-
 --
 -- String functions
 --
diff --git a/regress/expected/agtype.out b/regress/expected/agtype.out
index cada21f..9af214f 100644
--- a/regress/expected/agtype.out
+++ b/regress/expected/agtype.out
@@ -875,7 +875,7 @@ SELECT 3 % '3.14::numeric'::agtype;
 (1 row)
 
 --
--- Test overloaded agytype any functions and operators for NULL input 
+-- Test overloaded agytype any functions and operators for NULL input
 -- +, -, *, /, %, =, <>, <, >, <=, >=
 -- These should all return null
 SELECT agtype_any_add('null'::agtype, 1);
@@ -2486,6 +2486,102 @@ SELECT age_end_id(agtype_in('null'));
  
 (1 row)
 
+SELECT agtype_contains('{"id": 1}','{"id": 1}');
+ agtype_contains 
+-----------------
+ t
+(1 row)
+
+SELECT agtype_contains('{"id": 1}','{"id": 2}');
+ agtype_contains 
+-----------------
+ f
+(1 row)
+
+SELECT '{"id": 1}'::agtype @> '{"id": 1}';
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT '{"id": 1}'::agtype @> '{"id": 2}';
+ ?column? 
+----------
+ f
+(1 row)
+
+SELECT agtype_exists('{"id": 1}','id');
+ agtype_exists 
+---------------
+ t
+(1 row)
+
+SELECT agtype_exists('{"id": 1}','not_id');
+ agtype_exists 
+---------------
+ f
+(1 row)
+
+SELECT '{"id": 1}'::agtype ? 'id';
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT '{"id": 1}'::agtype ? 'not_id';
+ ?column? 
+----------
+ f
+(1 row)
+
+SELECT agtype_exists_any('{"id": 1}', array['id']);
+ agtype_exists_any 
+-------------------
+ t
+(1 row)
+
+SELECT agtype_exists_any('{"id": 1}', array['not_id']);
+ agtype_exists_any 
+-------------------
+ f
+(1 row)
+
+SELECT '{"id": 1}'::agtype ?| array['id'];
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT '{"id": 1}'::agtype ?| array['not_id'];
+ ?column? 
+----------
+ f
+(1 row)
+
+SELECT agtype_exists_all('{"id": 1}', array['id']);
+ agtype_exists_all 
+-------------------
+ t
+(1 row)
+
+SELECT agtype_exists_all('{"id": 1}', array['not_id']);
+ agtype_exists_all 
+-------------------
+ f
+(1 row)
+
+SELECT '{"id": 1}'::agtype ?& array['id'];
+ ?column? 
+----------
+ t
+(1 row)
+
+SELECT '{"id": 1}'::agtype ?& array['not_id'];
+ ?column? 
+----------
+ f
+(1 row)
+
 --
 -- Test STARTS WITH, ENDS WITH, and CONTAINS
 --
diff --git a/regress/expected/index.out b/regress/expected/index.out
index 09af778..05ed577 100644
--- a/regress/expected/index.out
+++ b/regress/expected/index.out
@@ -224,7 +224,7 @@ SELECT * FROM cypher('cypher_index', $$ MATCH(n) DETACH DELETE n $$) AS (a agtyp
 (0 rows)
 
 /*
- * Section 2: Indices to Improve Query Runtime
+ * Section 2: Graphid Indices to Improve Join Performance
  */
 SELECT create_graph('agload_test_graph');
 NOTICE:  graph "agload_test_graph" has been created
@@ -233,59 +233,34 @@ NOTICE:  graph "agload_test_graph" has been created
  
 (1 row)
 
-ALTER TABLE agload_test_graph._ag_label_vertex
-CLUSTER ON _ag_label_vertex_pkey;
-SELECT create_vlabel('agload_test_graph','Country');
-NOTICE:  VLabel "Country" has been created
- create_vlabel 
----------------
- 
-(1 row)
-
-SELECT load_labels_from_file('agload_test_graph', 'Country',
-    'age_load/countries.csv');
- load_labels_from_file 
------------------------
- 
-(1 row)
+SELECT * FROM cypher('agload_test_graph', $$
+    CREATE (us:Country {name: "United States"}),
+        (ca:Country {name: "Canada"}),
+        (mx:Country {name: "Mexico"}),
+        (us)<-[:has_city]-(:City {name:"New York", country_code:"US"}),
+        (us)<-[:has_city]-(:City {name:"San Fransisco", country_code:"US"}),
+        (us)<-[:has_city]-(:City {name:"Los Angeles", country_code:"US"}),
+        (us)<-[:has_city]-(:City {name:"Seattle", country_code:"US"}),
+        (ca)<-[:has_city]-(:City {name:"Vancouver", country_code:"CA"}),
+        (ca)<-[:has_city]-(:City {name:"Toroto", country_code:"CA"}),
+        (ca)<-[:has_city]-(:City {name:"Montreal", country_code:"CA"}),
+        (mx)<-[:has_city]-(:City {name:"Mexico City", country_code:"MX"}),
+        (mx)<-[:has_city]-(:City {name:"Monterrey", country_code:"MX"}),
+        (mx)<-[:has_city]-(:City {name:"Tijuana", country_code:"MX"})
+$$) as (n agtype);
+ n 
+---
+(0 rows)
 
 ALTER TABLE agload_test_graph."Country" ADD PRIMARY KEY (id);
 CREATE UNIQUE INDEX CONCURRENTLY cntry_id_idx ON agload_test_graph."Country" (id);
 ALTER TABLE agload_test_graph."Country"  CLUSTER ON cntry_id_idx;
-SELECT create_vlabel('agload_test_graph','City');
-NOTICE:  VLabel "City" has been created
- create_vlabel 
----------------
- 
-(1 row)
-
-SELECT load_labels_from_file('agload_test_graph', 'City',
-    'age_load/cities.csv');
- load_labels_from_file 
------------------------
- 
-(1 row)
-
 ALTER TABLE agload_test_graph."City"
 ADD PRIMARY KEY (id);
 CREATE UNIQUE INDEX city_id_idx
 ON agload_test_graph."City" (id);
 ALTER TABLE agload_test_graph."City"
 CLUSTER ON city_id_idx;
-SELECT create_elabel('agload_test_graph','has_city');
-NOTICE:  ELabel "has_city" has been created
- create_elabel 
----------------
- 
-(1 row)
-
-SELECT load_edges_from_file('agload_test_graph', 'has_city',
-     'age_load/edges.csv');
- load_edges_from_file 
-----------------------
- 
-(1 row)
-
 ALTER TABLE agload_test_graph.has_city
 ADD CONSTRAINT has_city_end_fk FOREIGN KEY (end_id)
 REFERENCES agload_test_graph."Country"(id) MATCH FULL;
@@ -304,7 +279,7 @@ SELECT COUNT(*) FROM cypher('agload_test_graph', $$
 $$) as (n agtype);
  count 
 -------
- 72485
+    10
 (1 row)
 
 SET enable_mergejoin = OFF;
@@ -316,7 +291,7 @@ SELECT COUNT(*) FROM cypher('agload_test_graph', $$
 $$) as (n agtype);
  count 
 -------
- 72485
+    10
 (1 row)
 
 SET enable_mergejoin = OFF;
@@ -328,9 +303,26 @@ SELECT COUNT(*) FROM cypher('agload_test_graph', $$
 $$) as (n agtype);
  count 
 -------
- 72485
+    10
 (1 row)
 
+--
+-- Section 3: Agtype GIN Indices to Improve WHERE clause Performance
+--
+CREATE INDEX load_city_gid_idx
+ON agload_test_graph."City" USING gin (properties);
+SELECT COUNT(*) FROM cypher('agload_test_graph', $$
+    MATCH (c:City {country_code: "AD"})
+    RETURN c
+$$) as (n agtype);
+ count 
+-------
+     0
+(1 row)
+
+--
+-- General Cleanup
+--
 SELECT drop_graph('cypher_index', true);
 NOTICE:  drop cascades to 3 other objects
 DETAIL:  drop cascades to table cypher_index._ag_label_vertex
@@ -347,8 +339,8 @@ NOTICE:  drop cascades to 5 other objects
 DETAIL:  drop cascades to table agload_test_graph._ag_label_vertex
 drop cascades to table agload_test_graph._ag_label_edge
 drop cascades to table agload_test_graph."Country"
-drop cascades to table agload_test_graph."City"
 drop cascades to table agload_test_graph.has_city
+drop cascades to table agload_test_graph."City"
 NOTICE:  graph "agload_test_graph" has been dropped
  drop_graph 
 ------------
diff --git a/regress/sql/agtype.sql b/regress/sql/agtype.sql
index 3103697..3ba69aa 100644
--- a/regress/sql/agtype.sql
+++ b/regress/sql/agtype.sql
@@ -267,7 +267,7 @@ SELECT 3 % '3.14'::agtype;
 SELECT 3 % '3.14::numeric'::agtype;
 
 --
--- Test overloaded agytype any functions and operators for NULL input 
+-- Test overloaded agytype any functions and operators for NULL input
 -- +, -, *, /, %, =, <>, <, >, <=, >=
 -- These should all return null
 SELECT agtype_any_add('null'::agtype, 1);
@@ -742,6 +742,31 @@ SELECT age_id(agtype_in('null'));
 SELECT age_start_id(agtype_in('null'));
 SELECT age_end_id(agtype_in('null'));
 
+SELECT agtype_contains('{"id": 1}','{"id": 1}');
+SELECT agtype_contains('{"id": 1}','{"id": 2}');
+
+SELECT '{"id": 1}'::agtype @> '{"id": 1}';
+SELECT '{"id": 1}'::agtype @> '{"id": 2}';
+
+SELECT agtype_exists('{"id": 1}','id');
+SELECT agtype_exists('{"id": 1}','not_id');
+
+SELECT '{"id": 1}'::agtype ? 'id';
+SELECT '{"id": 1}'::agtype ? 'not_id';
+
+SELECT agtype_exists_any('{"id": 1}', array['id']);
+SELECT agtype_exists_any('{"id": 1}', array['not_id']);
+
+SELECT '{"id": 1}'::agtype ?| array['id'];
+SELECT '{"id": 1}'::agtype ?| array['not_id'];
+
+
+SELECT agtype_exists_all('{"id": 1}', array['id']);
+SELECT agtype_exists_all('{"id": 1}', array['not_id']);
+
+SELECT '{"id": 1}'::agtype ?& array['id'];
+SELECT '{"id": 1}'::agtype ?& array['not_id'];
+
 --
 -- Test STARTS WITH, ENDS WITH, and CONTAINS
 --
diff --git a/regress/sql/index.sql b/regress/sql/index.sql
index 0ec3ab5..62695f3 100644
--- a/regress/sql/index.sql
+++ b/regress/sql/index.sql
@@ -127,28 +127,32 @@ SELECT * FROM cypher('cypher_index', $$ MATCH(n) MERGE (n)-[:e]->(:idx {i: n.i})
 --data cleanup
 SELECT * FROM cypher('cypher_index', $$ MATCH(n) DETACH DELETE n $$) AS (a agtype);
 
-
 /*
- * Section 2: Indices to Improve Query Runtime
+ * Section 2: Graphid Indices to Improve Join Performance
  */
 SELECT create_graph('agload_test_graph');
 
-ALTER TABLE agload_test_graph._ag_label_vertex
-CLUSTER ON _ag_label_vertex_pkey;
-
-SELECT create_vlabel('agload_test_graph','Country');
-SELECT load_labels_from_file('agload_test_graph', 'Country',
-    'age_load/countries.csv');
+SELECT * FROM cypher('agload_test_graph', $$
+    CREATE (us:Country {name: "United States"}),
+        (ca:Country {name: "Canada"}),
+        (mx:Country {name: "Mexico"}),
+        (us)<-[:has_city]-(:City {name:"New York", country_code:"US"}),
+        (us)<-[:has_city]-(:City {name:"San Fransisco", country_code:"US"}),
+        (us)<-[:has_city]-(:City {name:"Los Angeles", country_code:"US"}),
+        (us)<-[:has_city]-(:City {name:"Seattle", country_code:"US"}),
+        (ca)<-[:has_city]-(:City {name:"Vancouver", country_code:"CA"}),
+        (ca)<-[:has_city]-(:City {name:"Toroto", country_code:"CA"}),
+        (ca)<-[:has_city]-(:City {name:"Montreal", country_code:"CA"}),
+        (mx)<-[:has_city]-(:City {name:"Mexico City", country_code:"MX"}),
+        (mx)<-[:has_city]-(:City {name:"Monterrey", country_code:"MX"}),
+        (mx)<-[:has_city]-(:City {name:"Tijuana", country_code:"MX"})
+$$) as (n agtype);
 
 ALTER TABLE agload_test_graph."Country" ADD PRIMARY KEY (id);
+
 CREATE UNIQUE INDEX CONCURRENTLY cntry_id_idx ON agload_test_graph."Country" (id);
 ALTER TABLE agload_test_graph."Country"  CLUSTER ON cntry_id_idx;
 
-
-SELECT create_vlabel('agload_test_graph','City');
-SELECT load_labels_from_file('agload_test_graph', 'City',
-    'age_load/cities.csv');
-
 ALTER TABLE agload_test_graph."City"
 ADD PRIMARY KEY (id);
 
@@ -158,11 +162,6 @@ ON agload_test_graph."City" (id);
 ALTER TABLE agload_test_graph."City"
 CLUSTER ON city_id_idx;
 
-
-SELECT create_elabel('agload_test_graph','has_city');
-SELECT load_edges_from_file('agload_test_graph', 'has_city',
-     'age_load/edges.csv');
-
 ALTER TABLE agload_test_graph.has_city
 ADD CONSTRAINT has_city_end_fk FOREIGN KEY (end_id)
 REFERENCES agload_test_graph."Country"(id) MATCH FULL;
@@ -176,6 +175,7 @@ ON agload_test_graph.has_city (start_id);
 ALTER TABLE agload_test_graph."has_city"
 CLUSTER ON load_has_city_eid_idx;
 
+
 SET enable_mergejoin = ON;
 SET enable_hashjoin = OFF;
 SET enable_nestloop = OFF;
@@ -203,5 +203,19 @@ SELECT COUNT(*) FROM cypher('agload_test_graph', $$
     RETURN e
 $$) as (n agtype);
 
+--
+-- Section 3: Agtype GIN Indices to Improve WHERE clause Performance
+--
+CREATE INDEX load_city_gid_idx
+ON agload_test_graph."City" USING gin (properties);
+
+SELECT COUNT(*) FROM cypher('agload_test_graph', $$
+    MATCH (c:City {country_code: "AD"})
+    RETURN c
+$$) as (n agtype);
+
+--
+-- General Cleanup
+--
 SELECT drop_graph('cypher_index', true);
 SELECT drop_graph('agload_test_graph', true);
diff --git a/src/backend/executor/cypher_set.c b/src/backend/executor/cypher_set.c
index 9b37b3b..1bc74b7 100644
--- a/src/backend/executor/cypher_set.c
+++ b/src/backend/executor/cypher_set.c
@@ -149,8 +149,7 @@ static HeapTuple update_entity_tuple(ResultRelInfo *resultRelInfo,
         if (update_result != HeapTupleMayBeUpdated)
         {
             ereport(ERROR, (errcode(ERRCODE_INTERNAL_ERROR),
-                            errmsg("Entity failed to be updated: %i",
-                                   update_result)));
+                    errmsg("Entity failed to be updated: %i", update_result)));
         }
 
         // Insert index entries for the tuple
@@ -160,7 +159,6 @@ static HeapTuple update_entity_tuple(ResultRelInfo *resultRelInfo,
                                   false, NULL, NIL);
         }
     }
-
     ReleaseBuffer(buffer);
 
     estate->es_result_relation_info = saved_resultRelInfo;
@@ -293,13 +291,13 @@ static void update_all_paths(CustomScanState *node, graphid id,
         agtype *original_entity;
         agtype_value *original_entity_value;
 
-        // skip non agtype values
+        // skip nulls
         if (scanTupleSlot->tts_tupleDescriptor->attrs[i].atttypid != AGTYPEOID)
         {
             continue;
         }
 
-        // skip nulls
+        // skip non agtype values
         if (scanTupleSlot->tts_isnull[i])
         {
             continue;
@@ -432,7 +430,7 @@ static void process_update_list(CustomScanState *node)
          * NULL. It will be possible for a variable to be NULL when OPTIONAL
          * MATCH is implemented.
          */
-        if(update_item->remove_item)
+        if (update_item->remove_item)
         {
             remove_property = true;
         }
@@ -496,12 +494,12 @@ static void process_update_list(CustomScanState *node)
             slot = populate_edge_tts(slot, id, startid, endid,
                                      altered_properties);
         }
-	else
-	{
+        else
+        {
             ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
                             errmsg("age %s clause can only update vertex and edges",
                                    clause_name)));
-	}
+        }
 
         /* place the datum in its tuple table slot position. */
         scanTupleSlot->tts_values[update_item->entity_position - 1] = new_entity;
@@ -539,7 +537,7 @@ static void process_update_list(CustomScanState *node)
              * If the heap tuple still exists (It wasn't deleted between the
              * match and this SET/REMOVE) update the heap_tuple.
              */
-            if(HeapTupleIsValid(heap_tuple))
+            if (HeapTupleIsValid(heap_tuple))
             {
                 heap_tuple = update_entity_tuple(resultRelInfo, slot, estate,
                                                  heap_tuple);
diff --git a/src/backend/parser/cypher_clause.c b/src/backend/parser/cypher_clause.c
index 71fc41f..67dea9e 100644
--- a/src/backend/parser/cypher_clause.c
+++ b/src/backend/parser/cypher_clause.c
@@ -224,9 +224,9 @@ static transform_entity *
 make_transform_entity(cypher_parsestate *cpstate,
                       enum transform_entity_type type, Node *node, Expr *expr);
 static transform_entity *find_variable(cypher_parsestate *cpstate, char *name);
-static Node *create_property_constraint_function(cypher_parsestate *cpstate,
-                                                 transform_entity *entity,
-                                                 Node *property_constraints);
+static Node *create_property_constraints(cypher_parsestate *cpstate,
+                                         transform_entity *entity,
+                                         Node *property_constraints);
 static TargetEntry *findTarget(List *targetList, char *resname);
 static transform_entity *transform_VLE_edge_entity(cypher_parsestate *cpstate,
                                                    cypher_relationship *rel,
@@ -370,7 +370,7 @@ static List *make_target_list_from_join(ParseState *pstate,
 static Expr *add_volatile_wrapper(Expr *node);
 static FuncExpr *make_clause_func_expr(char *function_name,
                                        Node *clause_information);
-
+static char *get_entity_name(transform_entity *entity);
 /* for VLE support */
 static RangeTblEntry *transform_RangeFunction(cypher_parsestate *cpstate,
                                               RangeFunction *r);
@@ -808,7 +808,7 @@ transform_cypher_union_tree(cypher_parsestate *cpstate, cypher_clause *clause,
             {
                 TargetEntry *tle = (TargetEntry *) lfirst(tl);
 
-                if(!tle->resjunk)
+                if (!tle->resjunk)
                 {
                     *targetlist = lappend(*targetlist, tle);
                 }
@@ -2622,7 +2622,6 @@ static void transform_match_pattern(cypher_parsestate *cpstate, Query *query,
         }
     }
 
-
     query->rtable = cpstate->pstate.p_rtable;
     query->jointree = makeFromExpr(cpstate->pstate.p_joinlist, (Node *)expr);
 }
@@ -3227,41 +3226,46 @@ static transform_entity *find_variable(cypher_parsestate *cpstate, char *name)
     return NULL;
 }
 
-/*
- * Create a function to handle property constraints on an edge/vertex.
- * Since the property constraints might be a parameter, we cannot split
- * the property map into indvidual quals, this will be slightly inefficient,
- * but necessary to cover all possible situations.
- */
-static Node *create_property_constraint_function(cypher_parsestate *cpstate,
-                                                 transform_entity *entity,
-                                                 Node *property_constraints)
+static char *get_entity_name(transform_entity *entity)
 {
-    ParseState *pstate = (ParseState *)cpstate;
-    char *entity_name;
-    ColumnRef *cr;
-    FuncExpr *fexpr;
-    Oid func_oid;
-    Node *prop_expr, *const_expr;
-    RangeTblEntry *rte;
-
-    cr = makeNode(ColumnRef);
-
-    if (entity->type == ENT_EDGE)
+    if (entity->type == ENT_EDGE || entity->type == ENT_VLE_EDGE)
     {
-        entity_name = entity->entity.node->name;
+        return entity->entity.rel->name;
     }
     else if (entity->type == ENT_VERTEX)
     {
-        entity_name = entity->entity.rel->name;
+        return entity->entity.node->name;
     }
     else
     {
         ereport(ERROR,
                 (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-                        errmsg("cannot create a property constraint on non vertex or edge agtype")));
+                        errmsg("cannot get entity name from transform_entity type %i", entity->type)));
     }
 
+    return NULL;
+}
+
+/*
+ * Creates the Contains operator to process property contraints for a vertex/
+ * edge in a MATCH clause. creates the agtype @> with the enitity's properties
+ * on the right and the contraints in the MATCH clause on the left.
+ */
+static Node *create_property_constraints(cypher_parsestate *cpstate,
+                                         transform_entity *entity,
+                                         Node *property_constraints)
+{
+    ParseState *pstate = (ParseState *)cpstate;
+    char *entity_name;
+    ColumnRef *cr;
+    Node *prop_expr, *const_expr;
+    RangeTblEntry *rte;
+    Node *last_srf = pstate->p_last_srf;
+
+    cr = makeNode(ColumnRef);
+
+    entity_name = get_entity_name(entity);
+
     cr->fields = list_make2(makeString(entity_name), makeString("properties"));
 
     // use Postgres to get the properties' transform node
@@ -3279,13 +3283,8 @@ static Node *create_property_constraint_function(cypher_parsestate *cpstate,
     const_expr = transform_cypher_expr(cpstate, property_constraints,
                                        EXPR_KIND_WHERE);
 
-    func_oid = get_ag_func_oid("_property_constraint_check", 2, AGTYPEOID,
-                               AGTYPEOID);
-
-    fexpr = makeFuncExpr(func_oid, BOOLOID, list_make2(prop_expr, const_expr),
-                         InvalidOid, InvalidOid, COERCE_EXPLICIT_CALL);
-
-    return (Node *)fexpr;
+    return (Node *)make_op(pstate, list_make1(makeString("@>")), prop_expr,
+                           const_expr, last_srf, -1);
 }
 
 
@@ -3484,8 +3483,8 @@ static List *transform_match_entities(cypher_parsestate *cpstate, Query *query,
             {
                 Node *n = NULL;
 
-                n = create_property_constraint_function(cpstate, entity,
-                                                        node->props);
+                n = create_property_constraints(cpstate, entity, node->props);
+
                 cpstate->property_constraint_quals =
                     lappend(cpstate->property_constraint_quals, n);
             }
@@ -3542,9 +3541,8 @@ static List *transform_match_entities(cypher_parsestate *cpstate, Query *query,
 
                 if (rel->props)
                 {
-                    Node *n = create_property_constraint_function(cpstate,
-                                                                  entity,
-                                                                  rel->props);
+                    Node *n = create_property_constraints(cpstate, entity,
+                                                          rel->props);
                     cpstate->property_constraint_quals =
                         lappend(cpstate->property_constraint_quals, n);
                 }
@@ -4835,7 +4833,7 @@ transform_cypher_clause_as_subquery(cypher_parsestate *cpstate,
         checkNameSpaceConflicts(pstate, pstate->p_namespace, namespace);
     }
 
-    if(add_rte_to_query)
+    if (add_rte_to_query)
     {
         // all variables(attributes) from the previous clause(subquery) are visible
         addRTEtoQuery(pstate, rte, true, false, true);
diff --git a/src/backend/utils/adt/agtype.c b/src/backend/utils/adt/agtype.c
index b3277d6..0d38d9e 100644
--- a/src/backend/utils/adt/agtype.c
+++ b/src/backend/utils/adt/agtype.c
@@ -2722,7 +2722,8 @@ Datum agtype_to_int4_array(PG_FUNCTION_ARGS)
     agtype_iterator *agtype_iterator = agtype_iterator_init(&agtype_in->root);
     agtv_token = agtype_iterator_next(&agtype_iterator, &agtv, false);
 
-    if(agtv.type != AGTV_ARRAY) {
+    if (agtv.type != AGTV_ARRAY)
+    {
         cannot_cast_agtype_value(agtv.type, "int4[]");
     }
 
@@ -4071,25 +4072,6 @@ Datum agtype_typecast_path(PG_FUNCTION_ARGS)
     PG_RETURN_POINTER(agtype_value_to_agtype(path.res));
 }
 
-PG_FUNCTION_INFO_V1(_property_constraint_check);
-
-Datum _property_constraint_check(PG_FUNCTION_ARGS)
-{
-    agtype_iterator *constraint_it, *property_it;
-    agtype *properties, *constraints;
-
-    if (PG_ARGISNULL(0) || PG_ARGISNULL(1))
-        PG_RETURN_BOOL(false);
-
-    properties = AG_GET_ARG_AGTYPE_P(0);
-    constraints = AG_GET_ARG_AGTYPE_P(1);
-
-    constraint_it = agtype_iterator_init(&constraints->root);
-    property_it = agtype_iterator_init(&properties->root);
-
-    PG_RETURN_BOOL(agtype_deep_contains(&property_it, &constraint_it));
-}
-
 PG_FUNCTION_INFO_V1(age_id);
 
 Datum age_id(PG_FUNCTION_ARGS)
diff --git a/src/backend/utils/adt/agtype_gin.c b/src/backend/utils/adt/agtype_gin.c
new file mode 100644
index 0000000..fb28a51
--- /dev/null
+++ b/src/backend/utils/adt/agtype_gin.c
@@ -0,0 +1,553 @@
+/*
+ * For PostgreSQL Database Management System:
+ * (formerly known as Postgres, then as Postgres95)
+ *
+ * Portions Copyright (c) 1996-2010, The PostgreSQL Global Development Group
+ *
+ * Portions Copyright (c) 1994, The Regents of the University of California
+ *
+ * Permission to use, copy, modify, and distribute this software and its
+ * documentation for any purpose, without fee, and without a written agreement
+ * is hereby granted, provided that the above copyright notice and this
+ * paragraph and the following two paragraphs appear in all copies.
+ *
+ * IN NO EVENT SHALL THE UNIVERSITY OF CALIFORNIA BE LIABLE TO ANY PARTY FOR
+ * DIRECT, INDIRECT, SPECIAL, INCIDENTAL, OR CONSEQUENTIAL DAMAGES, INCLUDING
+ * LOST PROFITS, ARISING OUT OF THE USE OF THIS SOFTWARE AND ITS DOCUMENTATION,
+ * EVEN IF THE UNIVERSITY OF CALIFORNIA HAS BEEN ADVISED OF THE POSSIBILITY OF
+ * SUCH DAMAGE.
+ *
+ * THE UNIVERSITY OF CALIFORNIA SPECIFICALLY DISCLAIMS ANY WARRANTIES,
+ * INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND
+ * FITNESS FOR A PARTICULAR PURPOSE.
+ *
+ * THE SOFTWARE PROVIDED HEREUNDER IS ON AN "AS IS" BASIS, AND THE UNIVERSITY OF
+ * CALIFORNIA HAS NO OBLIGATIONS TO PROVIDE MAINTENANCE, SUPPORT, UPDATES,
+ * ENHANCEMENTS, OR MODIFICATIONS.
+ */
+
+#include "postgres.h"
+
+#include "access/gin.h"
+#include "access/hash.h"
+#include "access/stratnum.h"
+#include "catalog/pg_collation.h"
+#include "catalog/pg_type.h"
+#include "utils/builtins.h"
+#include "utils/varlena.h"
+
+#include "utils/agtype.h"
+
+typedef struct PathHashStack
+{
+    uint32 hash;
+    struct PathHashStack *parent;
+} PathHashStack;
+
+static Datum make_text_key(char flag, const char *str, int len);
+static Datum make_scalar_key(const agtype_value *scalar_val, bool is_key);
+
+#define MAXINT8LEN        25
+
+/*
+ *
+ * agtype_ops GIN opclass support functions
+ *
+ */
+/*
+ * Compares two keys (not indexed items!) and returns an integer less than zero,
+ * zero, or greater than zero, indicating whether the first key is less than,
+ * equal to, or greater than the second. NULL keys are never passed to this
+ * function.
+ */
+PG_FUNCTION_INFO_V1(gin_compare_agtype);
+Datum gin_compare_agtype(PG_FUNCTION_ARGS)
+{
+    text *arg1, *arg2;
+    int32 result;
+    char *a1p, *a2p;
+    int len1, len2;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1))
+    {
+        PG_RETURN_NULL();
+    }
+
+    arg1 = PG_GETARG_TEXT_PP(0);
+    arg2 = PG_GETARG_TEXT_PP(1);
+
+    a1p = VARDATA_ANY(arg1);
+    a2p = VARDATA_ANY(arg2);
+
+    len1 = VARSIZE_ANY_EXHDR(arg1);
+    len2 = VARSIZE_ANY_EXHDR(arg2);
+
+    /* Compare text as bttextcmp does, but always using C collation */
+    result = varstr_cmp(a1p, len1, a2p, len2, C_COLLATION_OID);
+
+    PG_FREE_IF_COPY(arg1, 0);
+    PG_FREE_IF_COPY(arg2, 1);
+
+    PG_RETURN_INT32(result);
+}
+
+/*
+ * Returns a palloc'd array of keys given an item to be indexed. The number of
+ * returned keys must be stored into *nkeys. The return value can be NULL if the
+ * item contains no keys.
+ */
+PG_FUNCTION_INFO_V1(gin_extract_agtype);
+Datum gin_extract_agtype(PG_FUNCTION_ARGS)
+{
+    agtype *agt;
+    int32 *nentries;
+    int total;
+    agtype_iterator *it;
+    agtype_value v;
+    agtype_iterator_token r;
+    int i = 0;
+    Datum *entries;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1))
+    {
+        PG_RETURN_POINTER(NULL);
+    }
+
+    agt = (agtype *) AG_GET_ARG_AGTYPE_P(0);
+    nentries = (int32 *) PG_GETARG_POINTER(1);
+    total = 2 * AGT_ROOT_COUNT(agt);
+
+    /* If the root level is empty, we certainly have no keys */
+    if (total == 0)
+    {
+        *nentries = 0;
+        PG_RETURN_POINTER(NULL);
+    }
+
+    /* Otherwise, use 2 * root count as initial estimate of result size */
+    entries = (Datum *) palloc(sizeof(Datum) * total);
+
+    it = agtype_iterator_init(&agt->root);
+
+    while ((r = agtype_iterator_next(&it, &v, false)) != WAGT_DONE)
+    {
+        /* Since we recurse into the object, we might need more space */
+        if (i >= total)
+        {
+            total *= 2;
+            entries = (Datum *) repalloc(entries, sizeof(Datum) * total);
+        }
+
+        switch (r)
+        {
+            case WAGT_KEY:
+                entries[i++] = make_scalar_key(&v, true);
+                break;
+            case WAGT_ELEM:
+                /* Pretend string array elements are keys */
+                entries[i++] = make_scalar_key(&v, (v.type == AGTV_STRING));
+                break;
+            case WAGT_VALUE:
+                entries[i++] = make_scalar_key(&v, false);
+                break;
+            default:
+                /* we can ignore structural items */
+                break;
+        }
+    }
+
+    *nentries = i;
+
+    PG_RETURN_POINTER(entries);
+}
+
+/*
+ * Returns a palloc'd array of keys given a value to be queried; that is, query
+ * is the value on the right-hand side of an indexable operator whose left-hand
+ * side is the indexed column. The number of returned keys must be stored into
+ * *nkeys. If any of the keys can be null, also palloc an array of *nkeys bool
+ * fields, store its address at *nullFlags, and set these null flags as needed.
+ * *nullFlags can be left NULL (its initial value) if all keys are non-null.
+ * The return value can be NULL if the query contains no keys.
+ *
+ * searchMode is an output argument that allows extractQuery to specify details
+ * about how the search will be done. If *searchMode is set to
+ * GIN_SEARCH_MODE_DEFAULT (which is the value it is initialized to before
+ * call), only items that match at least one of the returned keys are considered
+ * candidate matches. If *searchMode is set to GIN_SEARCH_MODE_ALL, then all
+ * non-null items in the index are considered candidate matches, whether they
+ * match any of the returned keys or not. This is only done when the contains
+ * or exists all strategy are used and the passed map is empty.
+ */
+PG_FUNCTION_INFO_V1(gin_extract_agtype_query);
+Datum gin_extract_agtype_query(PG_FUNCTION_ARGS)
+{
+    int32 *nentries;
+    StrategyNumber strategy;
+    int32 *searchMode;
+    Datum *entries;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1) ||
+        PG_ARGISNULL(2) || PG_ARGISNULL(6))
+    {
+        PG_RETURN_NULL();
+    }
+
+    nentries = (int32 *) PG_GETARG_POINTER(1);
+    strategy = PG_GETARG_UINT16(2);
+    searchMode = (int32 *) PG_GETARG_POINTER(6);
+
+    if (strategy == AGTYPE_CONTAINS_STRATEGY_NUMBER)
+    {
+        /* Query is a agtype, so just apply gin_extract_agtype... */
+        entries = (Datum *)
+            DatumGetPointer(DirectFunctionCall2(gin_extract_agtype,
+                                                PG_GETARG_DATUM(0),
+                                                PointerGetDatum(nentries)));
+        /* ...although "contains {}" requires a full index scan */
+        if (*nentries == 0)
+        {
+            *searchMode = GIN_SEARCH_MODE_ALL;
+        }
+    }
+    else if (strategy == AGTYPE_EXISTS_STRATEGY_NUMBER)
+    {
+        /* Query is a text string, which we treat as a key */
+        text *query = PG_GETARG_TEXT_PP(0);
+
+        *nentries = 1;
+        entries = (Datum *)palloc(sizeof(Datum));
+        entries[0] = make_text_key(AGT_GIN_FLAG_KEY, VARDATA_ANY(query),
+                                   VARSIZE_ANY_EXHDR(query));
+    }
+    else if (strategy == AGTYPE_EXISTS_ANY_STRATEGY_NUMBER ||
+             strategy == AGTYPE_EXISTS_ALL_STRATEGY_NUMBER)
+    {
+        /* Query is a text array; each element is treated as a key */
+        ArrayType *query = PG_GETARG_ARRAYTYPE_P(0);
+        Datum *key_datums;
+        bool *key_nulls;
+        int key_count;
+        int i, j;
+
+        deconstruct_array(query, TEXTOID, -1, false, 'i',
+                          &key_datums, &key_nulls, &key_count);
+
+        entries = (Datum *) palloc(sizeof(Datum) * key_count);
+
+        for (i = 0, j = 0; i < key_count; i++)
+        {
+            /* Nulls in the array are ignored */
+            if (key_nulls[i])
+            {
+                continue;
+            }
+
+            entries[j++] = make_text_key(AGT_GIN_FLAG_KEY,
+                                         VARDATA(key_datums[i]),
+                                         VARSIZE(key_datums[i]) - VARHDRSZ);
+        }
+
+        *nentries = j;
+        /* ExistsAll with no keys should match everything */
+        if (j == 0 && strategy == AGTYPE_EXISTS_ALL_STRATEGY_NUMBER)
+        {
+            *searchMode = GIN_SEARCH_MODE_ALL;
+        }
+    }
+    else
+    {
+        elog(ERROR, "unrecognized strategy number: %d", strategy);
+        entries = NULL;            /* keep compiler quiet */
+    }
+
+    PG_RETURN_POINTER(entries);
+}
+
+/*
+ * Returns true if an indexed item satisfies the query operator for the given
+ * strategy (or might satisfy it, if the recheck indication is returned). This
+ * function does not have direct access to the indexed item's value, since GIN
+ * does not store items explicitly. Rather, what is available is knowledge about
+ * which key values extracted from the query appear in a given indexed item. The
+ * check array has length nkeys, which is the same as the number of keys
+ * previously returned by gin_extract_agtype_query for this query datum. Each
+ * element of the check array is true if the indexed item contains the
+ * corresponding query key, i.e., if (check[i] == true) the i-th key of the
+ * gin_extract_agtype_query result array is present in the indexed item. The
+ * original query datum is passed in case the consistent method needs to consult
+ * it, and so are the queryKeys[] and nullFlags[] arrays previously returned by
+ * gin_extract_agtype_query.
+ *
+ * When extractQuery returns a null key in queryKeys[], the corresponding
+ * check[] element is true if the indexed item contains a null key; that is, the
+ * semantics of check[] are like IS NOT DISTINCT FROM. The consistent function
+ * can examine the corresponding nullFlags[] element if it needs to tell the
+ * difference between a regular value match and a null match.
+ *
+ * On success, *recheck should be set to true if the heap tuple needs to be
+ * rechecked against the query operator, or false if the index test is exact.
+ * That is, a false return value guarantees that the heap tuple does not match
+ * the query; a true return value with *recheck set to false guarantees that the
+ * heap tuple does match the query; and a true return value with *recheck set to
+ * true means that the heap tuple might match the query, so it needs to be
+ * fetched and rechecked by evaluating the query operator directly against the
+ * originally indexed item.
+ */
+PG_FUNCTION_INFO_V1(gin_consistent_agtype);
+Datum gin_consistent_agtype(PG_FUNCTION_ARGS)
+{
+    bool *check;
+    StrategyNumber strategy;
+    int32 nkeys;
+    bool *recheck;
+    bool res = true;
+    int32 i;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1) ||
+        PG_ARGISNULL(3) || PG_ARGISNULL(5))
+    {
+        PG_RETURN_NULL();
+    }
+
+    check = (bool *) PG_GETARG_POINTER(0);
+    strategy = PG_GETARG_UINT16(1);
+    nkeys = PG_GETARG_INT32(3);
+    recheck = (bool *) PG_GETARG_POINTER(5);
+
+    if (strategy == AGTYPE_CONTAINS_STRATEGY_NUMBER)
+    {
+        /*
+         * We must always recheck, since we can't tell from the index whether
+         * the positions of the matched items match the structure of the query
+         * object.  (Even if we could, we'd also have to worry about hashed
+         * keys and the index's failure to distinguish keys from string array
+         * elements.)  However, the tuple certainly doesn't match unless it
+         * contains all the query keys.
+         */
+        *recheck = true;
+        for (i = 0; i < nkeys; i++)
+        {
+            if (!check[i])
+            {
+                res = false;
+                break;
+            }
+        }
+    }
+    else if (strategy == AGTYPE_EXISTS_STRATEGY_NUMBER)
+    {
+        /*
+         * Although the key is certainly present in the index, we must recheck
+         * because (1) the key might be hashed, and (2) the index match might
+         * be for a key that's not at top level of the JSON object.  For (1),
+         * we could look at the query key to see if it's hashed and not
+         * recheck if not, but the index lacks enough info to tell about (2).
+         */
+        *recheck = true;
+        res = true;
+    }
+    else if (strategy == AGTYPE_EXISTS_ANY_STRATEGY_NUMBER)
+    {
+        /* As for plain exists, we must recheck */
+        *recheck = true;
+        res = true;
+    }
+    else if (strategy == AGTYPE_EXISTS_ALL_STRATEGY_NUMBER)
+    {
+        /* As for plain exists, we must recheck */
+        *recheck = true;
+        /* ... but unless all the keys are present, we can say "false" */
+        for (i = 0; i < nkeys; i++)
+        {
+            if (!check[i])
+            {
+                res = false;
+                break;
+            }
+        }
+    }
+    else
+    {
+        elog(ERROR, "unrecognized strategy number: %d", strategy);
+    }
+
+    PG_RETURN_BOOL(res);
+}
+
+/*
+ * gin_triconsistent_agtype is similar to gin_consistent_agtype, but instead of
+ * booleans in the check vector, there are three possible values for each key:
+ * GIN_TRUE, GIN_FALSE and GIN_MAYBE. GIN_FALSE and GIN_TRUE have the same
+ * meaning as regular boolean values, while GIN_MAYBE means that the presence of
+ * that key is not known. When GIN_MAYBE values are present, the function should
+ * only return GIN_TRUE if the item certainly matches whether or not the index
+ * item contains the corresponding query keys. Likewise, the function must
+ * return GIN_FALSE only if the item certainly does not match, whether or not it
+ * contains the GIN_MAYBE keys. If the result depends on the GIN_MAYBE entries,
+ * i.e., the match cannot be confirmed or refuted based on the known query keys,
+ * the function must return GIN_MAYBE.
+ *
+ * When there are no GIN_MAYBE values in the check vector, a GIN_MAYBE return
+ * value is the equivalent of setting the recheck flag in the boolean consistent
+ * function.
+ */
+PG_FUNCTION_INFO_V1(gin_triconsistent_agtype);
+Datum gin_triconsistent_agtype(PG_FUNCTION_ARGS)
+{
+    GinTernaryValue *check;
+    StrategyNumber strategy;
+    int32 nkeys;
+    GinTernaryValue res = GIN_MAYBE;
+    int32 i;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(3))
+    {
+        PG_RETURN_NULL();
+    }
+
+    check = (GinTernaryValue *)PG_GETARG_POINTER(0);
+    strategy = PG_GETARG_UINT16(1);
+    nkeys = PG_GETARG_INT32(3);
+
+    /*
+     * Note that we never return GIN_TRUE, only GIN_MAYBE or GIN_FALSE; this
+     * corresponds to always forcing recheck in the regular consistent
+     * function, for the reasons listed there.
+     */
+    if (strategy == AGTYPE_CONTAINS_STRATEGY_NUMBER ||
+        strategy == AGTYPE_EXISTS_ALL_STRATEGY_NUMBER)
+    {
+        /* All extracted keys must be present */
+        for (i = 0; i < nkeys; i++)
+        {
+            if (check[i] == GIN_FALSE)
+            {
+                res = GIN_FALSE;
+                break;
+            }
+        }
+    }
+    else if (strategy == AGTYPE_EXISTS_STRATEGY_NUMBER ||
+             strategy == AGTYPE_EXISTS_ANY_STRATEGY_NUMBER)
+    {
+        /* At least one extracted key must be present */
+        res = GIN_FALSE;
+        for (i = 0; i < nkeys; i++)
+        {
+            if (check[i] == GIN_TRUE || check[i] == GIN_MAYBE)
+            {
+                res = GIN_MAYBE;
+                break;
+            }
+        }
+    }
+    else
+    {
+        elog(ERROR, "unrecognized strategy number: %d", strategy);
+    }
+
+    PG_RETURN_GIN_TERNARY_VALUE(res);
+}
+
+/*
+ * Construct a agtype_ops GIN key from a flag byte and a textual representation
+ * (which need not be null-terminated).  This function is responsible
+ * for hashing overlength text representations; it will add the
+ * AGT_GIN_FLAG_HASHED bit to the flag value if it does that.
+ */
+static Datum make_text_key(char flag, const char *str, int len)
+{
+    text *item;
+    char hashbuf[10];
+
+    if (len > AGT_GIN_MAX_LENGTH)
+    {
+        uint32 hashval;
+
+        hashval = DatumGetUInt32(hash_any((const unsigned char *) str, len));
+        snprintf(hashbuf, sizeof(hashbuf), "%08x", hashval);
+        str = hashbuf;
+        len = 8;
+        flag |= AGT_GIN_FLAG_HASHED;
+    }
+
+    /*
+     * Now build the text Datum.  For simplicity we build a 4-byte-header
+     * varlena text Datum here, but we expect it will get converted to short
+     * header format when stored in the index.
+     */
+    item = (text *)palloc(VARHDRSZ + len + 1);
+    SET_VARSIZE(item, VARHDRSZ + len + 1);
+
+    *VARDATA(item) = flag;
+
+    memcpy(VARDATA(item) + 1, str, len);
+
+    return PointerGetDatum(item);
+}
+
+/*
+ * Create a textual representation of a agtype_value that will serve as a GIN
+ * key in a agtype_ops index.  is_key is true if the JsonbValue is a key,
+ * or if it is a string array element (since we pretend those are keys,
+ * see jsonb.h).
+ */
+static Datum make_scalar_key(const agtype_value *scalarVal, bool is_key)
+{
+    Datum item = 0;
+    char *cstr = NULL;
+    char buf[MAXINT8LEN + 1];
+    switch (scalarVal->type)
+    {
+        case AGTV_NULL:
+            Assert(!is_key);
+            item = make_text_key(AGT_GIN_FLAG_NULL, "", 0);
+            break;
+        case AGTV_INTEGER:
+            Assert(!is_key);
+            pg_lltoa(scalarVal->val.int_value, buf);
+            item = make_text_key(AGT_GIN_FLAG_NUM, buf, MAXINT8LEN + 1);
+            break;
+        case AGTV_FLOAT:
+            Assert(!is_key);
+            cstr = float8out_internal(scalarVal->val.float_value);
+            item = make_text_key(AGT_GIN_FLAG_NUM, cstr, strlen(cstr));
+            break;
+        case AGTV_BOOL:
+            Assert(!is_key);
+            item = make_text_key(AGT_GIN_FLAG_BOOL,
+                                 scalarVal->val.boolean ? "t" : "f", 1);
+            break;
+        case AGTV_NUMERIC:
+            Assert(!is_key);
+            /*
+             * A normalized textual representation, free of trailing zeroes,
+             * is required so that numerically equal values will produce equal
+             * strings.
+             *
+             * It isn't ideal that numerics are stored in a relatively bulky
+             * textual format.  However, it's a notationally convenient way of
+             * storing a "union" type in the GIN B-Tree, and indexing Jsonb
+             * strings takes precedence.
+             */
+            cstr = numeric_normalize(scalarVal->val.numeric);
+            item = make_text_key(AGT_GIN_FLAG_NUM, cstr, strlen(cstr));
+            pfree(cstr);
+            break;
+        case AGTV_STRING:
+            item = make_text_key(is_key ? AGT_GIN_FLAG_KEY : AGT_GIN_FLAG_STR,
+                                 scalarVal->val.string.val,
+                                 scalarVal->val.string.len);
+            break;
+        case AGTV_VERTEX:
+        case AGTV_EDGE:
+        case AGTV_PATH:
+            elog(ERROR, "agtype type: %d is not a scalar", scalarVal->type);
+        default:
+            elog(ERROR, "unrecognized agtype type: %d", scalarVal->type);
+            break;
+    }
+
+    return item;
+}
diff --git a/src/backend/utils/adt/agtype_ops.c b/src/backend/utils/adt/agtype_ops.c
index 7cf0541..9ee2180 100644
--- a/src/backend/utils/adt/agtype_ops.c
+++ b/src/backend/utils/adt/agtype_ops.c
@@ -25,6 +25,7 @@
 
 #include <math.h>
 
+#include "catalog/pg_type_d.h"
 #include "fmgr.h"
 #include "utils/builtins.h"
 #include "utils/numeric.h"
@@ -996,6 +997,165 @@ Datum agtype_any_ge(PG_FUNCTION_ARGS)
 }
 
 
+PG_FUNCTION_INFO_V1(agtype_contains);
+/*
+ * <@ operator for agtype. Returns true if the right agtype path/value entries
+ * contained at the top level within the left agtype value
+ */
+Datum agtype_contains(PG_FUNCTION_ARGS)
+{
+    agtype_iterator *constraint_it, *property_it;
+    agtype *properties, *constraints;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1))
+    {
+        PG_RETURN_BOOL(false);
+    }
+
+    properties = AG_GET_ARG_AGTYPE_P(0);
+    constraints = AG_GET_ARG_AGTYPE_P(1);
+
+    constraint_it = agtype_iterator_init(&constraints->root);
+    property_it = agtype_iterator_init(&properties->root);
+
+    PG_RETURN_BOOL(agtype_deep_contains(&property_it, &constraint_it));
+}
+
+
+PG_FUNCTION_INFO_V1(agtype_contained_by);
+/*
+ * <@ operator for agtype. Returns true if the left agtype path/value entries
+ * contained at the top level within the right agtype value
+ */
+Datum agtype_contained_by(PG_FUNCTION_ARGS)
+{
+    agtype_iterator *constraint_it, *property_it;
+    agtype *properties, *constraints;
+
+    if (PG_ARGISNULL(0) || PG_ARGISNULL(1))
+    {
+        PG_RETURN_BOOL(false);
+    }
+
+    properties = AG_GET_ARG_AGTYPE_P(0);
+    constraints = AG_GET_ARG_AGTYPE_P(1);
+
+    constraint_it = agtype_iterator_init(&constraints->root);
+    property_it = agtype_iterator_init(&properties->root);
+
+    PG_RETURN_BOOL(agtype_deep_contains(&constraint_it, &property_it));
+}
+
+PG_FUNCTION_INFO_V1(agtype_exists);
+/*
+ * ? operator for agtype. Returns true if the string exists as top-level keys
+ */
+Datum agtype_exists(PG_FUNCTION_ARGS)
+{
+    agtype *agt = AG_GET_ARG_AGTYPE_P(0);
+    text *key = PG_GETARG_TEXT_PP(1);
+    agtype_value aval;
+    agtype_value *v = NULL;
+
+    /*
+     * We only match Object keys (which are naturally always Strings), or
+     * string elements in arrays.  In particular, we do not match non-string
+     * scalar elements.  Existence of a key/element is only considered at the
+     * top level.  No recursion occurs.
+     */
+    aval.type = AGTV_STRING;
+    aval.val.string.val = VARDATA_ANY(key);
+    aval.val.string.len = VARSIZE_ANY_EXHDR(key);
+
+    v = find_agtype_value_from_container(&agt->root,
+                                         AGT_FOBJECT | AGT_FARRAY,
+                                         &aval);
+
+    PG_RETURN_BOOL(v != NULL);
+}
+
+PG_FUNCTION_INFO_V1(agtype_exists_any);
+/*
+ * ?| operator for agtype. Returns true if any of the array strings exist as
+ * top-level keys
+ */
+Datum agtype_exists_any(PG_FUNCTION_ARGS)
+{
+    agtype *agt = AG_GET_ARG_AGTYPE_P(0);
+    ArrayType *keys = PG_GETARG_ARRAYTYPE_P(1);
+    int i;
+    Datum *key_datums;
+    bool *key_nulls;
+    int elem_count;
+
+    deconstruct_array(keys, TEXTOID, -1, false, 'i', &key_datums, &key_nulls,
+                      &elem_count);
+
+    for (i = 0; i < elem_count; i++)
+    {
+        agtype_value strVal;
+
+        if (key_nulls[i])
+        {
+            continue;
+        }
+
+        strVal.type = AGTV_STRING;
+        strVal.val.string.val = VARDATA(key_datums[i]);
+        strVal.val.string.len = VARSIZE(key_datums[i]) - VARHDRSZ;
+
+        if (find_agtype_value_from_container(&agt->root,
+                                        AGT_FOBJECT | AGT_FARRAY,
+                                        &strVal) != NULL)
+        {
+            PG_RETURN_BOOL(true);
+        }
+    }
+
+    PG_RETURN_BOOL(false);
+}
+
+PG_FUNCTION_INFO_V1(agtype_exists_all);
+/*
+ * ?& operator for agtype. Returns true if all of the array strings exist as
+ * top-level keys
+ */
+Datum agtype_exists_all(PG_FUNCTION_ARGS)
+{
+    agtype *agt = AG_GET_ARG_AGTYPE_P(0);
+    ArrayType  *keys = PG_GETARG_ARRAYTYPE_P(1);
+    int i;
+    Datum *key_datums;
+    bool *key_nulls;
+    int elem_count;
+
+    deconstruct_array(keys, TEXTOID, -1, false, 'i', &key_datums, &key_nulls,
+                      &elem_count);
+
+    for (i = 0; i < elem_count; i++)
+    {
+        agtype_value strVal;
+
+        if (key_nulls[i])
+        {
+            continue;
+        }
+
+        strVal.type = AGTV_STRING;
+        strVal.val.string.val = VARDATA(key_datums[i]);
+        strVal.val.string.len = VARSIZE(key_datums[i]) - VARHDRSZ;
+
+        if (find_agtype_value_from_container(&agt->root,
+                                        AGT_FOBJECT | AGT_FARRAY,
+                                        &strVal) == NULL)
+        {
+            PG_RETURN_BOOL(false);
+        }
+    }
+
+    PG_RETURN_BOOL(true);
+}
+
 static agtype *agtype_concat(agtype *agt1, agtype *agt2)
 {
     agtype_parse_state *state = NULL;
diff --git a/src/backend/utils/load/ag_load_edges.c b/src/backend/utils/load/ag_load_edges.c
index 154f3c4..419f009 100644
--- a/src/backend/utils/load/ag_load_edges.c
+++ b/src/backend/utils/load/ag_load_edges.c
@@ -131,14 +131,26 @@ void edge_row_cb(int delim __attribute__((unused)), void *data)
 
 static int is_space(unsigned char c)
 {
-    if(c == CSV_SPACE || c == CSV_TAB) return 1;
-    return 0;
+    if (c == CSV_SPACE || c == CSV_TAB)
+    {
+        return 1;
+    }
+    else
+    {
+        return 0;
+    }
 }
 
 static int is_term(unsigned char c)
 {
-    if (c == CSV_CR || c == CSV_LF) return 1;
-    return 0;
+    if (c == CSV_CR || c == CSV_LF)
+    {
+        return 1;
+    }
+    else
+    {
+        return 0;
+    }
 }
 
 int create_edges_from_csv_file(char *file_path,
diff --git a/src/backend/utils/load/ag_load_labels.c b/src/backend/utils/load/ag_load_labels.c
index 216852d..c0ddf69 100644
--- a/src/backend/utils/load/ag_load_labels.c
+++ b/src/backend/utils/load/ag_load_labels.c
@@ -119,7 +119,9 @@ void vertex_row_cb(int delim __attribute__((unused)), void *data)
             cr->header_len[i] = cr->fields_len[i];
             cr->header[i] = strndup(cr->fields[i], cr->header_len[i]);
         }
-    } else {
+    }
+    else
+    {
         if (cr->id_field_exists)
         {
             label_id_int = strtol(cr->fields[0], NULL, 10);
@@ -157,15 +159,28 @@ void vertex_row_cb(int delim __attribute__((unused)), void *data)
 
 static int is_space(unsigned char c)
 {
-    if(c == CSV_SPACE || c == CSV_TAB) return 1;
-    return 0;
-}
+    if (c == CSV_SPACE || c == CSV_TAB)
+    {
+        return 1;
+    }
+    else
+    {
+        return 0;
+    }
 
+}
 static int is_term(unsigned char c)
 {
-    if (c == CSV_CR || c == CSV_LF) return 1;
-    return 0;
+    if (c == CSV_CR || c == CSV_LF)
+    {
+        return 1;
+    }
+    else
+    {
+        return 0;
+    }
 }
+
 int create_labels_from_csv_file(char *file_path,
                                 char *graph_name,
                                 Oid graph_id,