You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@age.apache.org by GitBox <gi...@apache.org> on 2021/12/06 13:11:34 UTC

[GitHub] [incubator-age] muhammadshoaib opened a new pull request #155: Create graph from files beta

muhammadshoaib opened a new pull request #155:
URL: https://github.com/apache/incubator-age/pull/155


   Two functions have been created that can be used as following to load vertex and edges from csv files. 
   
   ```
   CREATE FUNCTION ag_catalog.load_labels_from_file(graph_name name, label_name name, file_path text)
   
   CREATE FUNCTION ag_catalog.load_edges_from_file(graph_name name, label_name name, file_path text)
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] emotionbug commented on a change in pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
emotionbug commented on a change in pull request #155:
URL: https://github.com/apache/incubator-age/pull/155#discussion_r776545679



##########
File path: src/include/utils/load/ag_load_labels.h
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+#ifndef AG_LOAD_LABELS_H
+#define AG_LOAD_LABELS_H
+
+
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/xact.h"
+#include "catalog/dependency.h"
+#include "catalog/namespace.h"
+#include "catalog/objectaddress.h"
+#include "catalog/pg_class_d.h"
+#include "commands/defrem.h"
+#include "commands/sequence.h"
+#include "commands/tablecmds.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodes.h"
+#include "nodes/parsenodes.h"
+#include "nodes/pg_list.h"
+#include "nodes/plannodes.h"
+#include "nodes/primnodes.h"
+#include "nodes/value.h"
+#include "parser/parse_node.h"
+#include "parser/parser.h"
+#include "storage/lockdefs.h"
+#include "tcop/dest.h"
+#include "tcop/utility.h"
+#include "utils/acl.h"
+#include "utils/builtins.h"
+#include "utils/inval.h"
+#include "utils/lsyscache.h"
+#include "utils/rel.h"
+
+#include "catalog/ag_graph.h"
+#include "catalog/ag_label.h"
+#include "commands/label_commands.h"
+#include "utils/ag_cache.h"
+#include "utils/agtype.h"
+#include "utils/graphid.h"
+
+
+#define AGE_VERTIX 1
+#define AGE_EDGE 2

Review comment:
       + with counts struct.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] jrgemignani merged pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
jrgemignani merged pull request #155:
URL: https://github.com/apache/incubator-age/pull/155


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] JoshInnis commented on a change in pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
JoshInnis commented on a change in pull request #155:
URL: https://github.com/apache/incubator-age/pull/155#discussion_r767032582



##########
File path: src/backend/utils/load/ag_load_edges.c
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include "utils/load/csv.h"
+#include "utils/load/ag_load_edges.h"
+#include "utils/load/age_load.h"
+
+
+void edge_field_cb(void *field, size_t field_len, void *data) {
+
+    csv_edge_reader *cr = (csv_edge_reader*)data;
+    if (cr->error) {
+        cr->error = 1;
+        ereport(NOTICE,(errmsg("There is some unknown error")));
+    }
+
+    // check for space to store this field
+    if (cr->cur_field == cr->alloc) {
+        cr->alloc *= 2;
+        cr->fields = realloc(cr->fields, sizeof(char *) * cr->alloc);
+        cr->fields_len = realloc(cr->header, sizeof(size_t *) * cr->alloc);
+        if (cr->fields == NULL) {
+            cr->error = 1;
+            ereport(ERROR,
+                    (errmsg("field_cb: failed to reallocate %zu bytes\n", sizeof(char *) * cr->alloc)));
+        }
+    }
+    cr->fields_len[cr->cur_field] = field_len;
+    cr->curr_row_length += field_len;
+    cr->fields[cr->cur_field] = strndup((char*)field, field_len);
+    cr->cur_field += 1;
+}
+
+// Parser calls this function when it detects end of a row
+void edge_row_cb(int delim __attribute__((unused)), void *data) {
+
+    csv_edge_reader *cr = (csv_edge_reader*)data;
+
+    size_t i, n_fields;
+    int64 start_id_int;
+    graphid start_vertex_graph_id;
+    int start_vertex_type_id;
+
+    int64 end_id_int;
+    graphid end_vertex_graph_id;
+    int end_vertex_type_id;
+
+    graphid object_graph_id;
+
+    agtype* props = NULL;
+
+    n_fields = cr->cur_field;
+
+    if (cr->row == 0) {
+        cr->header_num = cr->cur_field;
+        cr->header_row_length = cr->curr_row_length;
+        cr->header_len = (size_t* )malloc(sizeof(size_t *) * cr->cur_field);
+        cr->header = malloc((sizeof (char*) * cr->cur_field));
+
+        for ( i = 0; i<cr->cur_field; i++) {
+            cr->header_len[i] = cr->fields_len[i];
+            cr->header[i] = strndup(cr->fields[i], cr->header_len[i]);
+        }
+    } else {
+        object_graph_id = make_graphid(cr->object_id, (int64)cr->row);
+
+        start_id_int = strtol(cr->fields[0], NULL, 10);
+        start_vertex_type_id = get_label_id(cr->fields[1], cr->graph_id);
+        end_id_int = strtol(cr->fields[2], NULL, 10);
+        end_vertex_type_id = get_label_id(cr->fields[3], cr->graph_id);
+
+        start_vertex_graph_id = make_graphid(start_vertex_type_id, start_id_int);
+        end_vertex_graph_id = make_graphid(end_vertex_type_id, end_id_int);
+
+        props = create_agtype_from_list_i(cr->header, cr->fields, n_fields, 3);
+
+        insert_edge_simple(cr->graph_id, cr->object_name, object_graph_id,
+                           start_vertex_graph_id, end_vertex_graph_id, props);
+
+    }
+
+    for (i = 0; i < n_fields; ++i) {
+        free(cr->fields[i]);
+    }
+
+    if (cr->error)
+        ereport(NOTICE,(errmsg("THere is some error")));
+
+
+    cr->cur_field = 0;
+    cr->curr_row_length = 0;
+    cr->row += 1;
+}
+
+static int is_space(unsigned char c) {
+    if(c == CSV_SPACE || c == CSV_TAB) return 1;
+    return 0;
+}
+
+/*

Review comment:
       This should be removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] emotionbug commented on a change in pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
emotionbug commented on a change in pull request #155:
URL: https://github.com/apache/incubator-age/pull/155#discussion_r776544909



##########
File path: regress/age_load/age_load_test.sql
##########
@@ -0,0 +1,34 @@
+DROP SCHEMA IF EXISTS agload_test_graph CASCADE ;

Review comment:
       It's not even a test case. Do we need this?

##########
File path: src/backend/utils/load/ag_load_edges.c
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include "utils/load/csv.h"
+#include "utils/load/ag_load_edges.h"
+#include "utils/load/age_load.h"
+
+
+void edge_field_cb(void *field, size_t field_len, void *data) {

Review comment:
       I don't know exact bracket rules in this project. but, never used like this.

##########
File path: src/include/utils/load/ag_load_labels.h
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+#ifndef AG_LOAD_LABELS_H
+#define AG_LOAD_LABELS_H
+
+
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/xact.h"
+#include "catalog/dependency.h"
+#include "catalog/namespace.h"
+#include "catalog/objectaddress.h"
+#include "catalog/pg_class_d.h"
+#include "commands/defrem.h"
+#include "commands/sequence.h"
+#include "commands/tablecmds.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodes.h"
+#include "nodes/parsenodes.h"
+#include "nodes/pg_list.h"
+#include "nodes/plannodes.h"
+#include "nodes/primnodes.h"
+#include "nodes/value.h"
+#include "parser/parse_node.h"
+#include "parser/parser.h"
+#include "storage/lockdefs.h"
+#include "tcop/dest.h"
+#include "tcop/utility.h"
+#include "utils/acl.h"
+#include "utils/builtins.h"
+#include "utils/inval.h"
+#include "utils/lsyscache.h"
+#include "utils/rel.h"
+
+#include "catalog/ag_graph.h"
+#include "catalog/ag_label.h"
+#include "commands/label_commands.h"
+#include "utils/ag_cache.h"
+#include "utils/agtype.h"
+#include "utils/graphid.h"
+
+
+#define AGE_VERTIX 1
+#define AGE_EDGE 2
+
+
+struct counts {
+    long unsigned fields;
+    long unsigned allvalues;
+    long unsigned rows;
+};
+
+typedef struct {
+    size_t row;
+    char **header;
+    size_t *header_len;
+    size_t header_num;
+    char **fields;
+    size_t *fields_len;
+    size_t alloc;
+    size_t cur_field;
+    int error;
+    size_t header_row_length;
+    size_t curr_row_length;
+    char *graph_name;
+    Oid graph_id;
+    char *object_name;
+    int object_id;
+    bool id_field_exists;
+} csv_vertex_reader;
+
+
+void vertex_field_cb(void *field, size_t field_len, void *data);
+void vertex_row_cb(int delim __attribute__((unused)), void *data);
+
+int create_labels_from_csv_file(char *file_path, char *graph_name, Oid graph_id,
+                                char *object_name, int object_id, bool id_field_exists);

Review comment:
       is it used?

##########
File path: src/include/utils/load/ag_load_labels.h
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+
+#ifndef AG_LOAD_LABELS_H
+#define AG_LOAD_LABELS_H
+
+
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/xact.h"
+#include "catalog/dependency.h"
+#include "catalog/namespace.h"
+#include "catalog/objectaddress.h"
+#include "catalog/pg_class_d.h"
+#include "commands/defrem.h"
+#include "commands/sequence.h"
+#include "commands/tablecmds.h"
+#include "miscadmin.h"
+#include "nodes/makefuncs.h"
+#include "nodes/nodes.h"
+#include "nodes/parsenodes.h"
+#include "nodes/pg_list.h"
+#include "nodes/plannodes.h"
+#include "nodes/primnodes.h"
+#include "nodes/value.h"
+#include "parser/parse_node.h"
+#include "parser/parser.h"
+#include "storage/lockdefs.h"
+#include "tcop/dest.h"
+#include "tcop/utility.h"
+#include "utils/acl.h"
+#include "utils/builtins.h"
+#include "utils/inval.h"
+#include "utils/lsyscache.h"
+#include "utils/rel.h"
+
+#include "catalog/ag_graph.h"
+#include "catalog/ag_label.h"
+#include "commands/label_commands.h"
+#include "utils/ag_cache.h"
+#include "utils/agtype.h"
+#include "utils/graphid.h"
+
+
+#define AGE_VERTIX 1
+#define AGE_EDGE 2

Review comment:
       is it used?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] muhammadshoaib commented on a change in pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
muhammadshoaib commented on a change in pull request #155:
URL: https://github.com/apache/incubator-age/pull/155#discussion_r776543232



##########
File path: src/include/utils/load/csv.h
##########
@@ -0,0 +1,108 @@
+//
+// Created by Shoaib on 12/5/2021.
+//
+
+/*
+libcsv - parse and write csv data
+Copyright (C) 2008-2021  Robert Gamble
+This library is free software; you can redistribute it and/or
+modify it under the terms of the GNU Lesser General Public
+License as published by the Free Software Foundation; either
+version 2.1 of the License, or (at your option) any later version.
+This library is distributed in the hope that it will be useful,
+but WITHOUT ANY WARRANTY; without even the implied warranty of
+MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+Lesser General Public License for more details.
+You should have received a copy of the GNU Lesser General Public
+License along with this library; if not, write to the Free Software
+Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301  USA
+*/
+
+#ifndef LIBCSV_H__
+#define LIBCSV_H__
+#include <stdlib.h>
+#include <stdio.h>
+
+#ifdef __cplusplus
+extern "C" {

Review comment:
       This is original library so I didn't changed anything in this one. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] JoshInnis commented on a change in pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
JoshInnis commented on a change in pull request #155:
URL: https://github.com/apache/incubator-age/pull/155#discussion_r767032582



##########
File path: src/backend/utils/load/ag_load_edges.c
##########
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include <stdio.h>
+#include <string.h>
+#include <errno.h>
+#include <stdlib.h>
+#include <unistd.h>
+
+#include "utils/load/csv.h"
+#include "utils/load/ag_load_edges.h"
+#include "utils/load/age_load.h"
+
+
+void edge_field_cb(void *field, size_t field_len, void *data) {
+
+    csv_edge_reader *cr = (csv_edge_reader*)data;
+    if (cr->error) {
+        cr->error = 1;
+        ereport(NOTICE,(errmsg("There is some unknown error")));
+    }
+
+    // check for space to store this field
+    if (cr->cur_field == cr->alloc) {
+        cr->alloc *= 2;
+        cr->fields = realloc(cr->fields, sizeof(char *) * cr->alloc);
+        cr->fields_len = realloc(cr->header, sizeof(size_t *) * cr->alloc);
+        if (cr->fields == NULL) {
+            cr->error = 1;
+            ereport(ERROR,
+                    (errmsg("field_cb: failed to reallocate %zu bytes\n", sizeof(char *) * cr->alloc)));
+        }
+    }
+    cr->fields_len[cr->cur_field] = field_len;
+    cr->curr_row_length += field_len;
+    cr->fields[cr->cur_field] = strndup((char*)field, field_len);
+    cr->cur_field += 1;
+}
+
+// Parser calls this function when it detects end of a row
+void edge_row_cb(int delim __attribute__((unused)), void *data) {
+
+    csv_edge_reader *cr = (csv_edge_reader*)data;
+
+    size_t i, n_fields;
+    int64 start_id_int;
+    graphid start_vertex_graph_id;
+    int start_vertex_type_id;
+
+    int64 end_id_int;
+    graphid end_vertex_graph_id;
+    int end_vertex_type_id;
+
+    graphid object_graph_id;
+
+    agtype* props = NULL;
+
+    n_fields = cr->cur_field;
+
+    if (cr->row == 0) {
+        cr->header_num = cr->cur_field;
+        cr->header_row_length = cr->curr_row_length;
+        cr->header_len = (size_t* )malloc(sizeof(size_t *) * cr->cur_field);
+        cr->header = malloc((sizeof (char*) * cr->cur_field));
+
+        for ( i = 0; i<cr->cur_field; i++) {
+            cr->header_len[i] = cr->fields_len[i];
+            cr->header[i] = strndup(cr->fields[i], cr->header_len[i]);
+        }
+    } else {
+        object_graph_id = make_graphid(cr->object_id, (int64)cr->row);
+
+        start_id_int = strtol(cr->fields[0], NULL, 10);
+        start_vertex_type_id = get_label_id(cr->fields[1], cr->graph_id);
+        end_id_int = strtol(cr->fields[2], NULL, 10);
+        end_vertex_type_id = get_label_id(cr->fields[3], cr->graph_id);
+
+        start_vertex_graph_id = make_graphid(start_vertex_type_id, start_id_int);
+        end_vertex_graph_id = make_graphid(end_vertex_type_id, end_id_int);
+
+        props = create_agtype_from_list_i(cr->header, cr->fields, n_fields, 3);
+
+        insert_edge_simple(cr->graph_id, cr->object_name, object_graph_id,
+                           start_vertex_graph_id, end_vertex_graph_id, props);
+
+    }
+
+    for (i = 0; i < n_fields; ++i) {
+        free(cr->fields[i]);
+    }
+
+    if (cr->error)
+        ereport(NOTICE,(errmsg("THere is some error")));
+
+
+    cr->cur_field = 0;
+    cr->curr_row_length = 0;
+    cr->row += 1;
+}
+
+static int is_space(unsigned char c) {
+    if(c == CSV_SPACE || c == CSV_TAB) return 1;
+    return 0;
+}
+
+/*

Review comment:
       This should be removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [incubator-age] emotionbug commented on a change in pull request #155: Create graph from files beta

Posted by GitBox <gi...@apache.org>.
emotionbug commented on a change in pull request #155:
URL: https://github.com/apache/incubator-age/pull/155#discussion_r776542642



##########
File path: src/include/utils/load/csv.h
##########
@@ -0,0 +1,108 @@
+//
+// Created by Shoaib on 12/5/2021.
+//
+
+/*
+libcsv - parse and write csv data
+Copyright (C) 2008-2021  Robert Gamble
+This library is free software; you can redistribute it and/or
+modify it under the terms of the GNU Lesser General Public
+License as published by the Free Software Foundation; either
+version 2.1 of the License, or (at your option) any later version.
+This library is distributed in the hope that it will be useful,
+but WITHOUT ANY WARRANTY; without even the implied warranty of
+MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+Lesser General Public License for more details.
+You should have received a copy of the GNU Lesser General Public
+License along with this library; if not, write to the Free Software
+Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA  02110-1301  USA
+*/
+
+#ifndef LIBCSV_H__
+#define LIBCSV_H__
+#include <stdlib.h>
+#include <stdio.h>
+
+#ifdef __cplusplus
+extern "C" {

Review comment:
       i think it need be indent.

##########
File path: regress/sql/age_load.sql
##########
@@ -0,0 +1,57 @@
+\! cp -r regress/age_load/data regress/instance/data/age_load

Review comment:
       is it need?

##########
File path: src/include/utils/load/csv.h
##########
@@ -0,0 +1,108 @@
+//
+// Created by Shoaib on 12/5/2021.

Review comment:
       remove this line




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@age.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org