You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/04/12 14:54:49 UTC

[1/4] arrow git commit: ARROW-810: [GLib] Remove io/ipc prefix

Repository: arrow
Updated Branches:
  refs/heads/master 5e5a5878d -> 6443b8287


http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/stream-writer.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/stream-writer.cpp b/c_glib/arrow-glib/stream-writer.cpp
new file mode 100644
index 0000000..016ce93
--- /dev/null
+++ b/c_glib/arrow-glib/stream-writer.cpp
@@ -0,0 +1,232 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/array.hpp>
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/record-batch.hpp>
+#include <arrow-glib/schema.hpp>
+
+#include <arrow-glib/output-stream.hpp>
+
+#include <arrow-glib/stream-writer.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: stream-writer
+ * @short_description: Stream writer class
+ *
+ * #GArrowStreamWriter is a class for sending data by stream based
+ * IPC.
+ */
+
+typedef struct GArrowStreamWriterPrivate_ {
+  std::shared_ptr<arrow::ipc::StreamWriter> stream_writer;
+} GArrowStreamWriterPrivate;
+
+enum {
+  PROP_0,
+  PROP_STREAM_WRITER
+};
+
+G_DEFINE_TYPE_WITH_PRIVATE(GArrowStreamWriter,
+                           garrow_stream_writer,
+                           G_TYPE_OBJECT);
+
+#define GARROW_STREAM_WRITER_GET_PRIVATE(obj)               \
+  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                           \
+                               GARROW_TYPE_STREAM_WRITER,   \
+                               GArrowStreamWriterPrivate))
+
+static void
+garrow_stream_writer_finalize(GObject *object)
+{
+  GArrowStreamWriterPrivate *priv;
+
+  priv = GARROW_STREAM_WRITER_GET_PRIVATE(object);
+
+  priv->stream_writer = nullptr;
+
+  G_OBJECT_CLASS(garrow_stream_writer_parent_class)->finalize(object);
+}
+
+static void
+garrow_stream_writer_set_property(GObject *object,
+                                    guint prop_id,
+                                    const GValue *value,
+                                    GParamSpec *pspec)
+{
+  GArrowStreamWriterPrivate *priv;
+
+  priv = GARROW_STREAM_WRITER_GET_PRIVATE(object);
+
+  switch (prop_id) {
+  case PROP_STREAM_WRITER:
+    priv->stream_writer =
+      *static_cast<std::shared_ptr<arrow::ipc::StreamWriter> *>(g_value_get_pointer(value));
+    break;
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_stream_writer_get_property(GObject *object,
+                                          guint prop_id,
+                                          GValue *value,
+                                          GParamSpec *pspec)
+{
+  switch (prop_id) {
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_stream_writer_init(GArrowStreamWriter *object)
+{
+}
+
+static void
+garrow_stream_writer_class_init(GArrowStreamWriterClass *klass)
+{
+  GObjectClass *gobject_class;
+  GParamSpec *spec;
+
+  gobject_class = G_OBJECT_CLASS(klass);
+
+  gobject_class->finalize     = garrow_stream_writer_finalize;
+  gobject_class->set_property = garrow_stream_writer_set_property;
+  gobject_class->get_property = garrow_stream_writer_get_property;
+
+  spec = g_param_spec_pointer("stream-writer",
+                              "ipc::StreamWriter",
+                              "The raw std::shared<arrow::ipc::StreamWriter> *",
+                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
+                                                       G_PARAM_CONSTRUCT_ONLY));
+  g_object_class_install_property(gobject_class, PROP_STREAM_WRITER, spec);
+}
+
+/**
+ * garrow_stream_writer_open:
+ * @sink: The output of the writer.
+ * @schema: The schema of the writer.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full): A newly opened
+ *   #GArrowStreamWriter or %NULL on error.
+ */
+GArrowStreamWriter *
+garrow_stream_writer_open(GArrowOutputStream *sink,
+                              GArrowSchema *schema,
+                              GError **error)
+{
+  std::shared_ptr<arrow::ipc::StreamWriter> arrow_stream_writer;
+  auto status =
+    arrow::ipc::StreamWriter::Open(garrow_output_stream_get_raw(sink).get(),
+                                 garrow_schema_get_raw(schema),
+                                 &arrow_stream_writer);
+  if (status.ok()) {
+    return garrow_stream_writer_new_raw(&arrow_stream_writer);
+  } else {
+    garrow_error_set(error, status, "[ipc][stream-writer][open]");
+    return NULL;
+  }
+}
+
+/**
+ * garrow_stream_writer_write_record_batch:
+ * @stream_writer: A #GArrowStreamWriter.
+ * @record_batch: The record batch to be written.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_stream_writer_write_record_batch(GArrowStreamWriter *stream_writer,
+                                            GArrowRecordBatch *record_batch,
+                                            GError **error)
+{
+  auto arrow_stream_writer =
+    garrow_stream_writer_get_raw(stream_writer);
+  auto arrow_record_batch =
+    garrow_record_batch_get_raw(record_batch);
+  auto arrow_record_batch_raw =
+    arrow_record_batch.get();
+
+  auto status = arrow_stream_writer->WriteRecordBatch(*arrow_record_batch_raw);
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[ipc][stream-writer][write-record-batch]");
+    return FALSE;
+  }
+}
+
+/**
+ * garrow_stream_writer_close:
+ * @stream_writer: A #GArrowStreamWriter.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_stream_writer_close(GArrowStreamWriter *stream_writer,
+                               GError **error)
+{
+  auto arrow_stream_writer =
+    garrow_stream_writer_get_raw(stream_writer);
+
+  auto status = arrow_stream_writer->Close();
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[ipc][stream-writer][close]");
+    return FALSE;
+  }
+}
+
+G_END_DECLS
+
+GArrowStreamWriter *
+garrow_stream_writer_new_raw(std::shared_ptr<arrow::ipc::StreamWriter> *arrow_stream_writer)
+{
+  auto stream_writer =
+    GARROW_STREAM_WRITER(g_object_new(GARROW_TYPE_STREAM_WRITER,
+                                        "stream-writer", arrow_stream_writer,
+                                        NULL));
+  return stream_writer;
+}
+
+std::shared_ptr<arrow::ipc::StreamWriter>
+garrow_stream_writer_get_raw(GArrowStreamWriter *stream_writer)
+{
+  GArrowStreamWriterPrivate *priv;
+
+  priv = GARROW_STREAM_WRITER_GET_PRIVATE(stream_writer);
+  return priv->stream_writer;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/stream-writer.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/stream-writer.h b/c_glib/arrow-glib/stream-writer.h
new file mode 100644
index 0000000..6e773f1
--- /dev/null
+++ b/c_glib/arrow-glib/stream-writer.h
@@ -0,0 +1,82 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow-glib/array.h>
+#include <arrow-glib/record-batch.h>
+#include <arrow-glib/schema.h>
+
+#include <arrow-glib/output-stream.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_STREAM_WRITER           \
+  (garrow_stream_writer_get_type())
+#define GARROW_STREAM_WRITER(obj)                           \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_STREAM_WRITER,    \
+                              GArrowStreamWriter))
+#define GARROW_STREAM_WRITER_CLASS(klass)                   \
+  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
+                           GARROW_TYPE_STREAM_WRITER,       \
+                           GArrowStreamWriterClass))
+#define GARROW_IS_STREAM_WRITER(obj)                        \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_STREAM_WRITER))
+#define GARROW_IS_STREAM_WRITER_CLASS(klass)                \
+  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
+                           GARROW_TYPE_STREAM_WRITER))
+#define GARROW_STREAM_WRITER_GET_CLASS(obj)                 \
+  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
+                             GARROW_TYPE_STREAM_WRITER,     \
+                             GArrowStreamWriterClass))
+
+typedef struct _GArrowStreamWriter         GArrowStreamWriter;
+typedef struct _GArrowStreamWriterClass    GArrowStreamWriterClass;
+
+/**
+ * GArrowStreamWriter:
+ *
+ * It wraps `arrow::ipc::StreamWriter`.
+ */
+struct _GArrowStreamWriter
+{
+  /*< private >*/
+  GObject parent_instance;
+};
+
+struct _GArrowStreamWriterClass
+{
+  GObjectClass parent_class;
+};
+
+GType garrow_stream_writer_get_type(void) G_GNUC_CONST;
+
+GArrowStreamWriter *garrow_stream_writer_open(GArrowOutputStream *sink,
+                                                     GArrowSchema *schema,
+                                                     GError **error);
+
+gboolean garrow_stream_writer_write_record_batch(GArrowStreamWriter *stream_writer,
+                                                     GArrowRecordBatch *record_batch,
+                                                     GError **error);
+gboolean garrow_stream_writer_close(GArrowStreamWriter *stream_writer,
+                                        GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/stream-writer.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/stream-writer.hpp b/c_glib/arrow-glib/stream-writer.hpp
new file mode 100644
index 0000000..994c83b
--- /dev/null
+++ b/c_glib/arrow-glib/stream-writer.hpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/api.h>
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/stream-writer.h>
+
+GArrowStreamWriter *garrow_stream_writer_new_raw(std::shared_ptr<arrow::ipc::StreamWriter> *arrow_stream_writer);
+std::shared_ptr<arrow::ipc::StreamWriter> garrow_stream_writer_get_raw(GArrowStreamWriter *stream_writer);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/writeable-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/writeable-file.cpp b/c_glib/arrow-glib/writeable-file.cpp
new file mode 100644
index 0000000..d0937ea
--- /dev/null
+++ b/c_glib/arrow-glib/writeable-file.cpp
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/writeable-file.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: writeable-file
+ * @title: GArrowWriteableFile
+ * @short_description: File output interface
+ *
+ * #GArrowWriteableFile is an interface for file output.
+ */
+
+G_DEFINE_INTERFACE(GArrowWriteableFile,
+                   garrow_writeable_file,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_writeable_file_default_init (GArrowWriteableFileInterface *iface)
+{
+}
+
+/**
+ * garrow_writeable_file_write_at:
+ * @writeable_file: A #GArrowWriteableFile.
+ * @position: The write start position.
+ * @data: (array length=n_bytes): The data to be written.
+ * @n_bytes: The number of bytes to be written.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_writeable_file_write_at(GArrowWriteableFile *writeable_file,
+                                  gint64 position,
+                                  const guint8 *data,
+                                  gint64 n_bytes,
+                                  GError **error)
+{
+  const auto arrow_writeable_file =
+    garrow_writeable_file_get_raw(writeable_file);
+
+  auto status = arrow_writeable_file->WriteAt(position, data, n_bytes);
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[io][writeable-file][write-at]");
+    return FALSE;
+  }
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::WriteableFile>
+garrow_writeable_file_get_raw(GArrowWriteableFile *writeable_file)
+{
+  auto *iface = GARROW_WRITEABLE_FILE_GET_IFACE(writeable_file);
+  return iface->get_raw(writeable_file);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/writeable-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/writeable-file.h b/c_glib/arrow-glib/writeable-file.h
new file mode 100644
index 0000000..7f4c186
--- /dev/null
+++ b/c_glib/arrow-glib/writeable-file.h
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_WRITEABLE_FILE           \
+  (garrow_writeable_file_get_type())
+#define GARROW_WRITEABLE_FILE(obj)                           \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_WRITEABLE_FILE,    \
+                              GArrowWriteableFile))
+#define GARROW_IS_WRITEABLE_FILE(obj)                        \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_WRITEABLE_FILE))
+#define GARROW_WRITEABLE_FILE_GET_IFACE(obj)                         \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                                 \
+                                 GARROW_TYPE_WRITEABLE_FILE,         \
+                                 GArrowWriteableFileInterface))
+
+typedef struct _GArrowWriteableFile          GArrowWriteableFile;
+typedef struct _GArrowWriteableFileInterface GArrowWriteableFileInterface;
+
+GType garrow_writeable_file_get_type(void) G_GNUC_CONST;
+
+gboolean garrow_writeable_file_write_at(GArrowWriteableFile *writeable_file,
+                                           gint64 position,
+                                           const guint8 *data,
+                                           gint64 n_bytes,
+                                           GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/writeable-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/writeable-file.hpp b/c_glib/arrow-glib/writeable-file.hpp
new file mode 100644
index 0000000..aa3cc50
--- /dev/null
+++ b/c_glib/arrow-glib/writeable-file.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/writeable-file.h>
+
+/**
+ * GArrowWriteableFile:
+ *
+ * It wraps `arrow::io::WriteableFile`.
+ */
+struct _GArrowWriteableFileInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::WriteableFile> (*get_raw)(GArrowWriteableFile *file);
+};
+
+std::shared_ptr<arrow::io::WriteableFile> garrow_writeable_file_get_raw(GArrowWriteableFile *writeable_file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/writeable.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/writeable.cpp b/c_glib/arrow-glib/writeable.cpp
new file mode 100644
index 0000000..6f4c630
--- /dev/null
+++ b/c_glib/arrow-glib/writeable.cpp
@@ -0,0 +1,106 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/writeable.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: writeable
+ * @title: GArrowWriteable
+ * @short_description: Output interface
+ *
+ * #GArrowWriteable is an interface for output. Output must be
+ * writeable.
+ */
+
+G_DEFINE_INTERFACE(GArrowWriteable,
+                   garrow_writeable,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_writeable_default_init (GArrowWriteableInterface *iface)
+{
+}
+
+/**
+ * garrow_writeable_write:
+ * @writeable: A #GArrowWriteable.
+ * @data: (array length=n_bytes): The data to be written.
+ * @n_bytes: The number of bytes to be written.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_writeable_write(GArrowWriteable *writeable,
+                          const guint8 *data,
+                          gint64 n_bytes,
+                          GError **error)
+{
+  const auto arrow_writeable = garrow_writeable_get_raw(writeable);
+
+  auto status = arrow_writeable->Write(data, n_bytes);
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[io][writeable][write]");
+    return FALSE;
+  }
+}
+
+/**
+ * garrow_writeable_flush:
+ * @writeable: A #GArrowWriteable.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * It ensures writing all data on memory to storage.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_writeable_flush(GArrowWriteable *writeable,
+                          GError **error)
+{
+  const auto arrow_writeable = garrow_writeable_get_raw(writeable);
+
+  auto status = arrow_writeable->Flush();
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[io][writeable][flush]");
+    return FALSE;
+  }
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::Writeable>
+garrow_writeable_get_raw(GArrowWriteable *writeable)
+{
+  auto *iface = GARROW_WRITEABLE_GET_IFACE(writeable);
+  return iface->get_raw(writeable);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/writeable.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/writeable.h b/c_glib/arrow-glib/writeable.h
new file mode 100644
index 0000000..66d6922
--- /dev/null
+++ b/c_glib/arrow-glib/writeable.h
@@ -0,0 +1,52 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_WRITEABLE                \
+  (garrow_writeable_get_type())
+#define GARROW_WRITEABLE(obj)                                \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_WRITEABLE,         \
+                              GArrowWriteable))
+#define GARROW_IS_WRITEABLE(obj)                             \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_WRITEABLE))
+#define GARROW_WRITEABLE_GET_IFACE(obj)                      \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
+                                 GARROW_TYPE_WRITEABLE,      \
+                                 GArrowWriteableInterface))
+
+typedef struct _GArrowWriteable          GArrowWriteable;
+typedef struct _GArrowWriteableInterface GArrowWriteableInterface;
+
+GType garrow_writeable_get_type(void) G_GNUC_CONST;
+
+gboolean garrow_writeable_write(GArrowWriteable *writeable,
+                                   const guint8 *data,
+                                   gint64 n_bytes,
+                                   GError **error);
+gboolean garrow_writeable_flush(GArrowWriteable *writeable,
+                                   GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/writeable.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/writeable.hpp b/c_glib/arrow-glib/writeable.hpp
new file mode 100644
index 0000000..2b398f8
--- /dev/null
+++ b/c_glib/arrow-glib/writeable.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/writeable.h>
+
+/**
+ * GArrowWriteableInterface:
+ *
+ * It wraps `arrow::io::Writeable`.
+ */
+struct _GArrowWriteableInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::Writeable> (*get_raw)(GArrowWriteable *file);
+};
+
+std::shared_ptr<arrow::io::Writeable> garrow_writeable_get_raw(GArrowWriteable *writeable);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/doc/reference/Makefile.am
----------------------------------------------------------------------
diff --git a/c_glib/doc/reference/Makefile.am b/c_glib/doc/reference/Makefile.am
index 116bc6c..d3389dc 100644
--- a/c_glib/doc/reference/Makefile.am
+++ b/c_glib/doc/reference/Makefile.am
@@ -26,7 +26,7 @@ SCAN_OPTIONS =						\
 	--deprecated-guards="GARROW_DISABLE_DEPRECATED"
 
 MKDB_OPTIONS =					\
-	--name-space=arrow			\
+	--name-space=garrow			\
 	--source-suffixes="c,cpp,h"
 
 HFILE_GLOB =					\

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/doc/reference/arrow-glib-docs.sgml
----------------------------------------------------------------------
diff --git a/c_glib/doc/reference/arrow-glib-docs.sgml b/c_glib/doc/reference/arrow-glib-docs.sgml
index 06a1936..396dce5 100644
--- a/c_glib/doc/reference/arrow-glib-docs.sgml
+++ b/c_glib/doc/reference/arrow-glib-docs.sgml
@@ -31,8 +31,8 @@
     </releaseinfo>
   </bookinfo>
 
-  <part id="arrow">
-    <title>GArrow</title>
+  <part id="data">
+    <title>Data</title>
     <chapter id="array">
       <title>Array</title>
       <xi:include href="xml/array.xml"/>
@@ -111,47 +111,47 @@
     </chapter>
   </part>
 
-  <part id="arrow-io">
-    <title>GArrowIO</title>
-    <chapter id="io-mode">
-      <title>Enums</title>
-      <xi:include href="xml/io-file-mode.xml"/>
+  <part id="io">
+    <title>IO</title>
+    <chapter id="mode">
+      <title>Mode</title>
+      <xi:include href="xml/file-mode.xml"/>
     </chapter>
-    <chapter id="io-input">
+    <chapter id="input">
       <title>Input</title>
-      <xi:include href="xml/io-readable.xml"/>
-      <xi:include href="xml/io-input-stream.xml"/>
-      <xi:include href="xml/io-random-access-file.xml"/>
+      <xi:include href="xml/readable.xml"/>
+      <xi:include href="xml/input-stream.xml"/>
+      <xi:include href="xml/random-access-file.xml"/>
     </chapter>
-    <chapter id="io-output">
+    <chapter id="output">
       <title>Output</title>
-      <xi:include href="xml/io-writeable.xml"/>
-      <xi:include href="xml/io-output-stream.xml"/>
-      <xi:include href="xml/io-writeable-file.xml"/>
-      <xi:include href="xml/io-file-output-stream.xml"/>
+      <xi:include href="xml/writeable.xml"/>
+      <xi:include href="xml/output-stream.xml"/>
+      <xi:include href="xml/writeable-file.xml"/>
+      <xi:include href="xml/file-output-stream.xml"/>
     </chapter>
-    <chapter id="io-input-output">
+    <chapter id="input-output">
       <title>Input and output</title>
-      <xi:include href="xml/io-file.xml"/>
-      <xi:include href="xml/io-memory-mapped-file.xml"/>
+      <xi:include href="xml/file.xml"/>
+      <xi:include href="xml/memory-mapped-file.xml"/>
     </chapter>
   </part>
 
-  <part id="arrow-ipc">
-    <title>GArrowIPC</title>
-    <chapter id="ipc-metadata">
-      <title>Enums</title>
-      <xi:include href="xml/ipc-metadata-version.xml"/>
+  <part id="ipc">
+    <title>IPC</title>
+    <chapter id="metadata">
+      <title>Metadata</title>
+      <xi:include href="xml/metadata-version.xml"/>
     </chapter>
-    <chapter id="ipc-reader">
+    <chapter id="reader">
       <title>Reader</title>
-      <xi:include href="xml/ipc-file-reader.xml"/>
-      <xi:include href="xml/ipc-stream-reader.xml"/>
+      <xi:include href="xml/file-reader.xml"/>
+      <xi:include href="xml/stream-reader.xml"/>
     </chapter>
-    <chapter id="ipc-writer">
-      <title>Input</title>
-      <xi:include href="xml/ipc-file-writer.xml"/>
-      <xi:include href="xml/ipc-stream-writer.xml"/>
+    <chapter id="writer">
+      <title>Writer</title>
+      <xi:include href="xml/file-writer.xml"/>
+      <xi:include href="xml/stream-writer.xml"/>
     </chapter>
   </part>
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/example/read-batch.c
----------------------------------------------------------------------
diff --git a/c_glib/example/read-batch.c b/c_glib/example/read-batch.c
index a55b085..dce96b8 100644
--- a/c_glib/example/read-batch.c
+++ b/c_glib/example/read-batch.c
@@ -87,14 +87,14 @@ int
 main(int argc, char **argv)
 {
   const char *input_path = "/tmp/batch.arrow";
-  GArrowIOMemoryMappedFile *input;
+  GArrowMemoryMappedFile *input;
   GError *error = NULL;
 
   if (argc > 1)
     input_path = argv[1];
-  input = garrow_io_memory_mapped_file_open(input_path,
-                                            GARROW_IO_FILE_MODE_READ,
-                                            &error);
+  input = garrow_memory_mapped_file_open(input_path,
+                                         GARROW_FILE_MODE_READ,
+                                         &error);
   if (!input) {
     g_print("failed to open file: %s\n", error->message);
     g_error_free(error);
@@ -102,10 +102,10 @@ main(int argc, char **argv)
   }
 
   {
-    GArrowIPCFileReader *reader;
+    GArrowFileReader *reader;
 
-    reader = garrow_ipc_file_reader_open(GARROW_IO_RANDOM_ACCESS_FILE(input),
-                                         &error);
+    reader = garrow_file_reader_open(GARROW_RANDOM_ACCESS_FILE(input),
+                                     &error);
     if (!reader) {
       g_print("failed to open file reader: %s\n", error->message);
       g_error_free(error);
@@ -116,12 +116,12 @@ main(int argc, char **argv)
     {
       guint i, n;
 
-      n = garrow_ipc_file_reader_get_n_record_batches(reader);
+      n = garrow_file_reader_get_n_record_batches(reader);
       for (i = 0; i < n; i++) {
         GArrowRecordBatch *record_batch;
 
         record_batch =
-          garrow_ipc_file_reader_get_record_batch(reader, i, &error);
+          garrow_file_reader_get_record_batch(reader, i, &error);
         if (!record_batch) {
           g_print("failed to open file reader: %s\n", error->message);
           g_error_free(error);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/example/read-stream.c
----------------------------------------------------------------------
diff --git a/c_glib/example/read-stream.c b/c_glib/example/read-stream.c
index c56942c..ba461e3 100644
--- a/c_glib/example/read-stream.c
+++ b/c_glib/example/read-stream.c
@@ -87,14 +87,14 @@ int
 main(int argc, char **argv)
 {
   const char *input_path = "/tmp/stream.arrow";
-  GArrowIOMemoryMappedFile *input;
+  GArrowMemoryMappedFile *input;
   GError *error = NULL;
 
   if (argc > 1)
     input_path = argv[1];
-  input = garrow_io_memory_mapped_file_open(input_path,
-                                            GARROW_IO_FILE_MODE_READ,
-                                            &error);
+  input = garrow_memory_mapped_file_open(input_path,
+                                         GARROW_FILE_MODE_READ,
+                                         &error);
   if (!input) {
     g_print("failed to open file: %s\n", error->message);
     g_error_free(error);
@@ -102,10 +102,10 @@ main(int argc, char **argv)
   }
 
   {
-    GArrowIPCStreamReader *reader;
+    GArrowStreamReader *reader;
 
-    reader = garrow_ipc_stream_reader_open(GARROW_IO_INPUT_STREAM(input),
-                                           &error);
+    reader = garrow_stream_reader_open(GARROW_INPUT_STREAM(input),
+                                       &error);
     if (!reader) {
       g_print("failed to open stream reader: %s\n", error->message);
       g_error_free(error);
@@ -117,7 +117,7 @@ main(int argc, char **argv)
       GArrowRecordBatch *record_batch;
 
       record_batch =
-        garrow_ipc_stream_reader_get_next_record_batch(reader, &error);
+        garrow_stream_reader_get_next_record_batch(reader, &error);
       if (error) {
         g_print("failed to get record batch: %s\n", error->message);
         g_error_free(error);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-file-output-stream.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-file-output-stream.rb b/c_glib/test/test-file-output-stream.rb
new file mode 100644
index 0000000..26737c0
--- /dev/null
+++ b/c_glib/test/test-file-output-stream.rb
@@ -0,0 +1,38 @@
+# 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.
+
+class TestFileOutputStream < Test::Unit::TestCase
+  sub_test_case(".open") do
+    def test_create
+      tempfile = Tempfile.open("arrow-io-file-output-stream")
+      tempfile.write("Hello")
+      tempfile.close
+      file = Arrow::FileOutputStream.open(tempfile.path, false)
+      file.close
+      assert_equal("", File.read(tempfile.path))
+    end
+
+    def test_append
+      tempfile = Tempfile.open("arrow-io-file-output-stream")
+      tempfile.write("Hello")
+      tempfile.close
+      file = Arrow::FileOutputStream.open(tempfile.path, true)
+      file.close
+      assert_equal("Hello", File.read(tempfile.path))
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-file-writer.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-file-writer.rb b/c_glib/test/test-file-writer.rb
new file mode 100644
index 0000000..31c094d
--- /dev/null
+++ b/c_glib/test/test-file-writer.rb
@@ -0,0 +1,45 @@
+# 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.
+
+class TestFileWriter < Test::Unit::TestCase
+  def test_write_record_batch
+    tempfile = Tempfile.open("arrow-ipc-file-writer")
+    output = Arrow::FileOutputStream.open(tempfile.path, false)
+    begin
+      field = Arrow::Field.new("enabled", Arrow::BooleanDataType.new)
+      schema = Arrow::Schema.new([field])
+      file_writer = Arrow::FileWriter.open(output, schema)
+      begin
+        record_batch = Arrow::RecordBatch.new(schema, 0, [])
+        file_writer.write_record_batch(record_batch)
+      ensure
+        file_writer.close
+      end
+    ensure
+      output.close
+    end
+
+    input = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      file_reader = Arrow::FileReader.open(input)
+      assert_equal(["enabled"],
+                   file_reader.schema.fields.collect(&:name))
+    ensure
+      input.close
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-io-file-output-stream.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-io-file-output-stream.rb b/c_glib/test/test-io-file-output-stream.rb
deleted file mode 100644
index e35a183..0000000
--- a/c_glib/test/test-io-file-output-stream.rb
+++ /dev/null
@@ -1,38 +0,0 @@
-# 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.
-
-class TestIOFileOutputStream < Test::Unit::TestCase
-  sub_test_case(".open") do
-    def test_create
-      tempfile = Tempfile.open("arrow-io-file-output-stream")
-      tempfile.write("Hello")
-      tempfile.close
-      file = Arrow::IOFileOutputStream.open(tempfile.path, false)
-      file.close
-      assert_equal("", File.read(tempfile.path))
-    end
-
-    def test_append
-      tempfile = Tempfile.open("arrow-io-file-output-stream")
-      tempfile.write("Hello")
-      tempfile.close
-      file = Arrow::IOFileOutputStream.open(tempfile.path, true)
-      file.close
-      assert_equal("Hello", File.read(tempfile.path))
-    end
-  end
-end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-io-memory-mapped-file.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-io-memory-mapped-file.rb b/c_glib/test/test-io-memory-mapped-file.rb
deleted file mode 100644
index 197d188..0000000
--- a/c_glib/test/test-io-memory-mapped-file.rb
+++ /dev/null
@@ -1,138 +0,0 @@
-# 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.
-
-class TestIOMemoryMappedFile < Test::Unit::TestCase
-  def test_open
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      buffer = " " * 5
-      file.read(buffer)
-      assert_equal("Hello", buffer)
-    ensure
-      file.close
-    end
-  end
-
-  def test_size
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      assert_equal(5, file.size)
-    ensure
-      file.close
-    end
-  end
-
-  def test_read
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello World")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      buffer = " " * 5
-      _success, n_read_bytes = file.read(buffer)
-      assert_equal("Hello", buffer.byteslice(0, n_read_bytes))
-    ensure
-      file.close
-    end
-  end
-
-  def test_read_at
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello World")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      buffer = " " * 5
-      _success, n_read_bytes = file.read_at(6, buffer)
-      assert_equal("World", buffer.byteslice(0, n_read_bytes))
-    ensure
-      file.close
-    end
-  end
-
-  def test_write
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :readwrite)
-    begin
-      file.write("World")
-    ensure
-      file.close
-    end
-    assert_equal("World", File.read(tempfile.path))
-  end
-
-  def test_write_at
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :readwrite)
-    begin
-      file.write_at(2, "rld")
-    ensure
-      file.close
-    end
-    assert_equal("Herld", File.read(tempfile.path))
-  end
-
-  def test_flush
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :readwrite)
-    begin
-      file.write("World")
-      file.flush
-      assert_equal("World", File.read(tempfile.path))
-    ensure
-      file.close
-    end
-  end
-
-  def test_tell
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello World")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      buffer = " " * 5
-      file.read(buffer)
-      assert_equal(5, file.tell)
-    ensure
-      file.close
-    end
-  end
-
-  def test_mode
-    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
-    tempfile.write("Hello World")
-    tempfile.close
-    file = Arrow::IOMemoryMappedFile.open(tempfile.path, :readwrite)
-    begin
-      assert_equal(Arrow::IOFileMode::READWRITE, file.mode)
-    ensure
-      file.close
-    end
-  end
-end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-ipc-file-writer.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-ipc-file-writer.rb b/c_glib/test/test-ipc-file-writer.rb
deleted file mode 100644
index 1c33ccc..0000000
--- a/c_glib/test/test-ipc-file-writer.rb
+++ /dev/null
@@ -1,45 +0,0 @@
-# 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.
-
-class TestIPCFileWriter < Test::Unit::TestCase
-  def test_write_record_batch
-    tempfile = Tempfile.open("arrow-ipc-file-writer")
-    output = Arrow::IOFileOutputStream.open(tempfile.path, false)
-    begin
-      field = Arrow::Field.new("enabled", Arrow::BooleanDataType.new)
-      schema = Arrow::Schema.new([field])
-      file_writer = Arrow::IPCFileWriter.open(output, schema)
-      begin
-        record_batch = Arrow::RecordBatch.new(schema, 0, [])
-        file_writer.write_record_batch(record_batch)
-      ensure
-        file_writer.close
-      end
-    ensure
-      output.close
-    end
-
-    input = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      file_reader = Arrow::IPCFileReader.open(input)
-      assert_equal(["enabled"],
-                   file_reader.schema.fields.collect(&:name))
-    ensure
-      input.close
-    end
-  end
-end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-ipc-stream-writer.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-ipc-stream-writer.rb b/c_glib/test/test-ipc-stream-writer.rb
deleted file mode 100644
index 78bb4a7..0000000
--- a/c_glib/test/test-ipc-stream-writer.rb
+++ /dev/null
@@ -1,53 +0,0 @@
-# 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.
-
-class TestIPCStreamWriter < Test::Unit::TestCase
-  include Helper::Buildable
-
-  def test_write_record_batch
-    tempfile = Tempfile.open("arrow-ipc-stream-writer")
-    output = Arrow::IOFileOutputStream.open(tempfile.path, false)
-    begin
-      field = Arrow::Field.new("enabled", Arrow::BooleanDataType.new)
-      schema = Arrow::Schema.new([field])
-      stream_writer = Arrow::IPCStreamWriter.open(output, schema)
-      begin
-        columns = [
-          build_boolean_array([true]),
-        ]
-        record_batch = Arrow::RecordBatch.new(schema, 1, columns)
-        stream_writer.write_record_batch(record_batch)
-      ensure
-        stream_writer.close
-      end
-    ensure
-      output.close
-    end
-
-    input = Arrow::IOMemoryMappedFile.open(tempfile.path, :read)
-    begin
-      stream_reader = Arrow::IPCStreamReader.open(input)
-      assert_equal(["enabled"],
-                   stream_reader.schema.fields.collect(&:name))
-      assert_equal(true,
-                   stream_reader.next_record_batch.get_column(0).get_value(0))
-      assert_nil(stream_reader.next_record_batch)
-    ensure
-      input.close
-    end
-  end
-end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-memory-mapped-file.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-memory-mapped-file.rb b/c_glib/test/test-memory-mapped-file.rb
new file mode 100644
index 0000000..e78d07a
--- /dev/null
+++ b/c_glib/test/test-memory-mapped-file.rb
@@ -0,0 +1,138 @@
+# 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.
+
+class TestMemoryMappedFile < Test::Unit::TestCase
+  def test_open
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      buffer = " " * 5
+      file.read(buffer)
+      assert_equal("Hello", buffer)
+    ensure
+      file.close
+    end
+  end
+
+  def test_size
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      assert_equal(5, file.size)
+    ensure
+      file.close
+    end
+  end
+
+  def test_read
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello World")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      buffer = " " * 5
+      _success, n_read_bytes = file.read(buffer)
+      assert_equal("Hello", buffer.byteslice(0, n_read_bytes))
+    ensure
+      file.close
+    end
+  end
+
+  def test_read_at
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello World")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      buffer = " " * 5
+      _success, n_read_bytes = file.read_at(6, buffer)
+      assert_equal("World", buffer.byteslice(0, n_read_bytes))
+    ensure
+      file.close
+    end
+  end
+
+  def test_write
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :readwrite)
+    begin
+      file.write("World")
+    ensure
+      file.close
+    end
+    assert_equal("World", File.read(tempfile.path))
+  end
+
+  def test_write_at
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :readwrite)
+    begin
+      file.write_at(2, "rld")
+    ensure
+      file.close
+    end
+    assert_equal("Herld", File.read(tempfile.path))
+  end
+
+  def test_flush
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :readwrite)
+    begin
+      file.write("World")
+      file.flush
+      assert_equal("World", File.read(tempfile.path))
+    ensure
+      file.close
+    end
+  end
+
+  def test_tell
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello World")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      buffer = " " * 5
+      file.read(buffer)
+      assert_equal(5, file.tell)
+    ensure
+      file.close
+    end
+  end
+
+  def test_mode
+    tempfile = Tempfile.open("arrow-io-memory-mapped-file")
+    tempfile.write("Hello World")
+    tempfile.close
+    file = Arrow::MemoryMappedFile.open(tempfile.path, :readwrite)
+    begin
+      assert_equal(Arrow::FileMode::READWRITE, file.mode)
+    ensure
+      file.close
+    end
+  end
+end

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/test/test-stream-writer.rb
----------------------------------------------------------------------
diff --git a/c_glib/test/test-stream-writer.rb b/c_glib/test/test-stream-writer.rb
new file mode 100644
index 0000000..306115e
--- /dev/null
+++ b/c_glib/test/test-stream-writer.rb
@@ -0,0 +1,53 @@
+# 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.
+
+class TestStreamWriter < Test::Unit::TestCase
+  include Helper::Buildable
+
+  def test_write_record_batch
+    tempfile = Tempfile.open("arrow-ipc-stream-writer")
+    output = Arrow::FileOutputStream.open(tempfile.path, false)
+    begin
+      field = Arrow::Field.new("enabled", Arrow::BooleanDataType.new)
+      schema = Arrow::Schema.new([field])
+      stream_writer = Arrow::StreamWriter.open(output, schema)
+      begin
+        columns = [
+          build_boolean_array([true]),
+        ]
+        record_batch = Arrow::RecordBatch.new(schema, 1, columns)
+        stream_writer.write_record_batch(record_batch)
+      ensure
+        stream_writer.close
+      end
+    ensure
+      output.close
+    end
+
+    input = Arrow::MemoryMappedFile.open(tempfile.path, :read)
+    begin
+      stream_reader = Arrow::StreamReader.open(input)
+      assert_equal(["enabled"],
+                   stream_reader.schema.fields.collect(&:name))
+      assert_equal(true,
+                   stream_reader.next_record_batch.get_column(0).get_value(0))
+      assert_nil(stream_reader.next_record_batch)
+    ensure
+      input.close
+    end
+  end
+end


[2/4] arrow git commit: ARROW-810: [GLib] Remove io/ipc prefix

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-file-writer.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-file-writer.hpp b/c_glib/arrow-glib/ipc-file-writer.hpp
deleted file mode 100644
index b8ae113..0000000
--- a/c_glib/arrow-glib/ipc-file-writer.hpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/api.h>
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/ipc-file-writer.h>
-
-GArrowIPCFileWriter *garrow_ipc_file_writer_new_raw(std::shared_ptr<arrow::ipc::FileWriter> *arrow_file_writer);
-arrow::ipc::FileWriter *garrow_ipc_file_writer_get_raw(GArrowIPCFileWriter *file_writer);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-metadata-version.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-metadata-version.cpp b/c_glib/arrow-glib/ipc-metadata-version.cpp
deleted file mode 100644
index f591f29..0000000
--- a/c_glib/arrow-glib/ipc-metadata-version.cpp
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow-glib/ipc-metadata-version.hpp>
-
-/**
- * SECTION: ipc-metadata-version
- * @title: GArrowIPCMetadataVersion
- * @short_description: Metadata version mapgging between Arrow and arrow-glib
- *
- * #GArrowIPCMetadataVersion provides metadata versions corresponding
- * to `arrow::ipc::MetadataVersion` values.
- */
-
-GArrowIPCMetadataVersion
-garrow_ipc_metadata_version_from_raw(arrow::ipc::MetadataVersion version)
-{
-  switch (version) {
-  case arrow::ipc::MetadataVersion::V1:
-    return GARROW_IPC_METADATA_VERSION_V1;
-  case arrow::ipc::MetadataVersion::V2:
-    return GARROW_IPC_METADATA_VERSION_V2;
-  case arrow::ipc::MetadataVersion::V3:
-    return GARROW_IPC_METADATA_VERSION_V3;
-  default:
-    return GARROW_IPC_METADATA_VERSION_V3;
-  }
-}
-
-arrow::ipc::MetadataVersion
-garrow_ipc_metadata_version_to_raw(GArrowIPCMetadataVersion version)
-{
-  switch (version) {
-  case GARROW_IPC_METADATA_VERSION_V1:
-    return arrow::ipc::MetadataVersion::V1;
-  case GARROW_IPC_METADATA_VERSION_V2:
-    return arrow::ipc::MetadataVersion::V2;
-  case GARROW_IPC_METADATA_VERSION_V3:
-    return arrow::ipc::MetadataVersion::V3;
-  default:
-    return arrow::ipc::MetadataVersion::V3;
-  }
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-metadata-version.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-metadata-version.h b/c_glib/arrow-glib/ipc-metadata-version.h
deleted file mode 100644
index 20defdb..0000000
--- a/c_glib/arrow-glib/ipc-metadata-version.h
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-/**
- * GArrowIPCMetadataVersion:
- * @GARROW_IPC_METADATA_VERSION_V1: Version 1.
- * @GARROW_IPC_METADATA_VERSION_V2: Version 2.
- * @GARROW_IPC_METADATA_VERSION_V3: Version 3.
- *
- * They are corresponding to `arrow::ipc::MetadataVersion::type`
- * values.
- */
-typedef enum {
-  GARROW_IPC_METADATA_VERSION_V1,
-  GARROW_IPC_METADATA_VERSION_V2,
-  GARROW_IPC_METADATA_VERSION_V3
-} GArrowIPCMetadataVersion;
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-metadata-version.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-metadata-version.hpp b/c_glib/arrow-glib/ipc-metadata-version.hpp
deleted file mode 100644
index 229565f..0000000
--- a/c_glib/arrow-glib/ipc-metadata-version.hpp
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/ipc-metadata-version.h>
-
-GArrowIPCMetadataVersion garrow_ipc_metadata_version_from_raw(arrow::ipc::MetadataVersion version);
-arrow::ipc::MetadataVersion garrow_ipc_metadata_version_to_raw(GArrowIPCMetadataVersion version);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-stream-reader.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-stream-reader.cpp b/c_glib/arrow-glib/ipc-stream-reader.cpp
deleted file mode 100644
index 4804784..0000000
--- a/c_glib/arrow-glib/ipc-stream-reader.cpp
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/record-batch.hpp>
-#include <arrow-glib/schema.hpp>
-
-#include <arrow-glib/io-input-stream.hpp>
-
-#include <arrow-glib/ipc-metadata-version.hpp>
-#include <arrow-glib/ipc-stream-reader.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: ipc-stream-reader
- * @short_description: Stream reader class
- *
- * #GArrowIPCStreamReader is a class for receiving data by stream
- * based IPC.
- */
-
-typedef struct GArrowIPCStreamReaderPrivate_ {
-  std::shared_ptr<arrow::ipc::StreamReader> stream_reader;
-} GArrowIPCStreamReaderPrivate;
-
-enum {
-  PROP_0,
-  PROP_STREAM_READER
-};
-
-G_DEFINE_TYPE_WITH_PRIVATE(GArrowIPCStreamReader,
-                           garrow_ipc_stream_reader,
-                           G_TYPE_OBJECT);
-
-#define GARROW_IPC_STREAM_READER_GET_PRIVATE(obj)               \
-  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                           \
-                               GARROW_IPC_TYPE_STREAM_READER,   \
-                               GArrowIPCStreamReaderPrivate))
-
-static void
-garrow_ipc_stream_reader_finalize(GObject *object)
-{
-  GArrowIPCStreamReaderPrivate *priv;
-
-  priv = GARROW_IPC_STREAM_READER_GET_PRIVATE(object);
-
-  priv->stream_reader = nullptr;
-
-  G_OBJECT_CLASS(garrow_ipc_stream_reader_parent_class)->finalize(object);
-}
-
-static void
-garrow_ipc_stream_reader_set_property(GObject *object,
-                                    guint prop_id,
-                                    const GValue *value,
-                                    GParamSpec *pspec)
-{
-  GArrowIPCStreamReaderPrivate *priv;
-
-  priv = GARROW_IPC_STREAM_READER_GET_PRIVATE(object);
-
-  switch (prop_id) {
-  case PROP_STREAM_READER:
-    priv->stream_reader =
-      *static_cast<std::shared_ptr<arrow::ipc::StreamReader> *>(g_value_get_pointer(value));
-    break;
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_ipc_stream_reader_get_property(GObject *object,
-                                          guint prop_id,
-                                          GValue *value,
-                                          GParamSpec *pspec)
-{
-  switch (prop_id) {
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_ipc_stream_reader_init(GArrowIPCStreamReader *object)
-{
-}
-
-static void
-garrow_ipc_stream_reader_class_init(GArrowIPCStreamReaderClass *klass)
-{
-  GObjectClass *gobject_class;
-  GParamSpec *spec;
-
-  gobject_class = G_OBJECT_CLASS(klass);
-
-  gobject_class->finalize     = garrow_ipc_stream_reader_finalize;
-  gobject_class->set_property = garrow_ipc_stream_reader_set_property;
-  gobject_class->get_property = garrow_ipc_stream_reader_get_property;
-
-  spec = g_param_spec_pointer("stream-reader",
-                              "ipc::StreamReader",
-                              "The raw std::shared<arrow::ipc::StreamReader> *",
-                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
-                                                       G_PARAM_CONSTRUCT_ONLY));
-  g_object_class_install_property(gobject_class, PROP_STREAM_READER, spec);
-}
-
-/**
- * garrow_ipc_stream_reader_open:
- * @stream: The stream to be read.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full): A newly opened
- *   #GArrowIPCStreamReader or %NULL on error.
- */
-GArrowIPCStreamReader *
-garrow_ipc_stream_reader_open(GArrowIOInputStream *stream,
-                              GError **error)
-{
-  std::shared_ptr<arrow::ipc::StreamReader> arrow_stream_reader;
-  auto status =
-    arrow::ipc::StreamReader::Open(garrow_io_input_stream_get_raw(stream),
-                                   &arrow_stream_reader);
-  if (status.ok()) {
-    return garrow_ipc_stream_reader_new_raw(&arrow_stream_reader);
-  } else {
-    garrow_error_set(error, status, "[ipc][stream-reader][open]");
-    return NULL;
-  }
-}
-
-/**
- * garrow_ipc_stream_reader_get_schema:
- * @stream_reader: A #GArrowIPCStreamReader.
- *
- * Returns: (transfer full): The schema in the stream.
- */
-GArrowSchema *
-garrow_ipc_stream_reader_get_schema(GArrowIPCStreamReader *stream_reader)
-{
-  auto arrow_stream_reader =
-    garrow_ipc_stream_reader_get_raw(stream_reader);
-  auto arrow_schema = arrow_stream_reader->schema();
-  return garrow_schema_new_raw(&arrow_schema);
-}
-
-/**
- * garrow_ipc_stream_reader_get_next_record_batch:
- * @stream_reader: A #GArrowIPCStreamReader.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full):
- *   The next record batch in the stream or %NULL on end of stream.
- */
-GArrowRecordBatch *
-garrow_ipc_stream_reader_get_next_record_batch(GArrowIPCStreamReader *stream_reader,
-                                               GError **error)
-{
-  auto arrow_stream_reader =
-    garrow_ipc_stream_reader_get_raw(stream_reader);
-  std::shared_ptr<arrow::RecordBatch> arrow_record_batch;
-  auto status = arrow_stream_reader->GetNextRecordBatch(&arrow_record_batch);
-
-  if (status.ok()) {
-    if (arrow_record_batch == nullptr) {
-      return NULL;
-    } else {
-      return garrow_record_batch_new_raw(&arrow_record_batch);
-    }
-  } else {
-    garrow_error_set(error, status, "[ipc][stream-reader][get-next-record-batch]");
-    return NULL;
-  }
-}
-
-G_END_DECLS
-
-GArrowIPCStreamReader *
-garrow_ipc_stream_reader_new_raw(std::shared_ptr<arrow::ipc::StreamReader> *arrow_stream_reader)
-{
-  auto stream_reader =
-    GARROW_IPC_STREAM_READER(g_object_new(GARROW_IPC_TYPE_STREAM_READER,
-                                          "stream-reader", arrow_stream_reader,
-                                          NULL));
-  return stream_reader;
-}
-
-std::shared_ptr<arrow::ipc::StreamReader>
-garrow_ipc_stream_reader_get_raw(GArrowIPCStreamReader *stream_reader)
-{
-  GArrowIPCStreamReaderPrivate *priv;
-
-  priv = GARROW_IPC_STREAM_READER_GET_PRIVATE(stream_reader);
-  return priv->stream_reader;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-stream-reader.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-stream-reader.h b/c_glib/arrow-glib/ipc-stream-reader.h
deleted file mode 100644
index 993cd85..0000000
--- a/c_glib/arrow-glib/ipc-stream-reader.h
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow-glib/record-batch.h>
-#include <arrow-glib/schema.h>
-
-#include <arrow-glib/io-input-stream.h>
-
-#include <arrow-glib/ipc-metadata-version.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IPC_TYPE_STREAM_READER           \
-  (garrow_ipc_stream_reader_get_type())
-#define GARROW_IPC_STREAM_READER(obj)                           \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IPC_TYPE_STREAM_READER,    \
-                              GArrowIPCStreamReader))
-#define GARROW_IPC_STREAM_READER_CLASS(klass)                   \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
-                           GARROW_IPC_TYPE_STREAM_READER,       \
-                           GArrowIPCStreamReaderClass))
-#define GARROW_IPC_IS_STREAM_READER(obj)                        \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IPC_TYPE_STREAM_READER))
-#define GARROW_IPC_IS_STREAM_READER_CLASS(klass)                \
-  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
-                           GARROW_IPC_TYPE_STREAM_READER))
-#define GARROW_IPC_STREAM_READER_GET_CLASS(obj)                 \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_IPC_TYPE_STREAM_READER,     \
-                             GArrowIPCStreamReaderClass))
-
-typedef struct _GArrowIPCStreamReader         GArrowIPCStreamReader;
-typedef struct _GArrowIPCStreamReaderClass    GArrowIPCStreamReaderClass;
-
-/**
- * GArrowIPCStreamReader:
- *
- * It wraps `arrow::ipc::StreamReader`.
- */
-struct _GArrowIPCStreamReader
-{
-  /*< private >*/
-  GObject parent_instance;
-};
-
-struct _GArrowIPCStreamReaderClass
-{
-  GObjectClass parent_class;
-};
-
-GType garrow_ipc_stream_reader_get_type(void) G_GNUC_CONST;
-
-GArrowIPCStreamReader *garrow_ipc_stream_reader_open(GArrowIOInputStream *stream,
-                                                     GError **error);
-
-GArrowSchema *garrow_ipc_stream_reader_get_schema(GArrowIPCStreamReader *stream_reader);
-GArrowRecordBatch *garrow_ipc_stream_reader_get_next_record_batch(GArrowIPCStreamReader *stream_reader,
-                                                                  GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-stream-reader.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-stream-reader.hpp b/c_glib/arrow-glib/ipc-stream-reader.hpp
deleted file mode 100644
index a35bdab..0000000
--- a/c_glib/arrow-glib/ipc-stream-reader.hpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/api.h>
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/ipc-stream-reader.h>
-
-GArrowIPCStreamReader *garrow_ipc_stream_reader_new_raw(std::shared_ptr<arrow::ipc::StreamReader> *arrow_stream_reader);
-std::shared_ptr<arrow::ipc::StreamReader> garrow_ipc_stream_reader_get_raw(GArrowIPCStreamReader *stream_reader);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-stream-writer.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-stream-writer.cpp b/c_glib/arrow-glib/ipc-stream-writer.cpp
deleted file mode 100644
index e2455a4..0000000
--- a/c_glib/arrow-glib/ipc-stream-writer.cpp
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/array.hpp>
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/record-batch.hpp>
-#include <arrow-glib/schema.hpp>
-
-#include <arrow-glib/io-output-stream.hpp>
-
-#include <arrow-glib/ipc-stream-writer.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: ipc-stream-writer
- * @short_description: Stream writer class
- *
- * #GArrowIPCStreamWriter is a class for sending data by stream based
- * IPC.
- */
-
-typedef struct GArrowIPCStreamWriterPrivate_ {
-  std::shared_ptr<arrow::ipc::StreamWriter> stream_writer;
-} GArrowIPCStreamWriterPrivate;
-
-enum {
-  PROP_0,
-  PROP_STREAM_WRITER
-};
-
-G_DEFINE_TYPE_WITH_PRIVATE(GArrowIPCStreamWriter,
-                           garrow_ipc_stream_writer,
-                           G_TYPE_OBJECT);
-
-#define GARROW_IPC_STREAM_WRITER_GET_PRIVATE(obj)               \
-  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                           \
-                               GARROW_IPC_TYPE_STREAM_WRITER,   \
-                               GArrowIPCStreamWriterPrivate))
-
-static void
-garrow_ipc_stream_writer_finalize(GObject *object)
-{
-  GArrowIPCStreamWriterPrivate *priv;
-
-  priv = GARROW_IPC_STREAM_WRITER_GET_PRIVATE(object);
-
-  priv->stream_writer = nullptr;
-
-  G_OBJECT_CLASS(garrow_ipc_stream_writer_parent_class)->finalize(object);
-}
-
-static void
-garrow_ipc_stream_writer_set_property(GObject *object,
-                                    guint prop_id,
-                                    const GValue *value,
-                                    GParamSpec *pspec)
-{
-  GArrowIPCStreamWriterPrivate *priv;
-
-  priv = GARROW_IPC_STREAM_WRITER_GET_PRIVATE(object);
-
-  switch (prop_id) {
-  case PROP_STREAM_WRITER:
-    priv->stream_writer =
-      *static_cast<std::shared_ptr<arrow::ipc::StreamWriter> *>(g_value_get_pointer(value));
-    break;
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_ipc_stream_writer_get_property(GObject *object,
-                                          guint prop_id,
-                                          GValue *value,
-                                          GParamSpec *pspec)
-{
-  switch (prop_id) {
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_ipc_stream_writer_init(GArrowIPCStreamWriter *object)
-{
-}
-
-static void
-garrow_ipc_stream_writer_class_init(GArrowIPCStreamWriterClass *klass)
-{
-  GObjectClass *gobject_class;
-  GParamSpec *spec;
-
-  gobject_class = G_OBJECT_CLASS(klass);
-
-  gobject_class->finalize     = garrow_ipc_stream_writer_finalize;
-  gobject_class->set_property = garrow_ipc_stream_writer_set_property;
-  gobject_class->get_property = garrow_ipc_stream_writer_get_property;
-
-  spec = g_param_spec_pointer("stream-writer",
-                              "ipc::StreamWriter",
-                              "The raw std::shared<arrow::ipc::StreamWriter> *",
-                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
-                                                       G_PARAM_CONSTRUCT_ONLY));
-  g_object_class_install_property(gobject_class, PROP_STREAM_WRITER, spec);
-}
-
-/**
- * garrow_ipc_stream_writer_open:
- * @sink: The output of the writer.
- * @schema: The schema of the writer.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full): A newly opened
- *   #GArrowIPCStreamWriter or %NULL on error.
- */
-GArrowIPCStreamWriter *
-garrow_ipc_stream_writer_open(GArrowIOOutputStream *sink,
-                              GArrowSchema *schema,
-                              GError **error)
-{
-  std::shared_ptr<arrow::ipc::StreamWriter> arrow_stream_writer;
-  auto status =
-    arrow::ipc::StreamWriter::Open(garrow_io_output_stream_get_raw(sink).get(),
-                                 garrow_schema_get_raw(schema),
-                                 &arrow_stream_writer);
-  if (status.ok()) {
-    return garrow_ipc_stream_writer_new_raw(&arrow_stream_writer);
-  } else {
-    garrow_error_set(error, status, "[ipc][stream-writer][open]");
-    return NULL;
-  }
-}
-
-/**
- * garrow_ipc_stream_writer_write_record_batch:
- * @stream_writer: A #GArrowIPCStreamWriter.
- * @record_batch: The record batch to be written.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_ipc_stream_writer_write_record_batch(GArrowIPCStreamWriter *stream_writer,
-                                            GArrowRecordBatch *record_batch,
-                                            GError **error)
-{
-  auto arrow_stream_writer =
-    garrow_ipc_stream_writer_get_raw(stream_writer);
-  auto arrow_record_batch =
-    garrow_record_batch_get_raw(record_batch);
-  auto arrow_record_batch_raw =
-    arrow_record_batch.get();
-
-  auto status = arrow_stream_writer->WriteRecordBatch(*arrow_record_batch_raw);
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[ipc][stream-writer][write-record-batch]");
-    return FALSE;
-  }
-}
-
-/**
- * garrow_ipc_stream_writer_close:
- * @stream_writer: A #GArrowIPCStreamWriter.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_ipc_stream_writer_close(GArrowIPCStreamWriter *stream_writer,
-                               GError **error)
-{
-  auto arrow_stream_writer =
-    garrow_ipc_stream_writer_get_raw(stream_writer);
-
-  auto status = arrow_stream_writer->Close();
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[ipc][stream-writer][close]");
-    return FALSE;
-  }
-}
-
-G_END_DECLS
-
-GArrowIPCStreamWriter *
-garrow_ipc_stream_writer_new_raw(std::shared_ptr<arrow::ipc::StreamWriter> *arrow_stream_writer)
-{
-  auto stream_writer =
-    GARROW_IPC_STREAM_WRITER(g_object_new(GARROW_IPC_TYPE_STREAM_WRITER,
-                                        "stream-writer", arrow_stream_writer,
-                                        NULL));
-  return stream_writer;
-}
-
-std::shared_ptr<arrow::ipc::StreamWriter>
-garrow_ipc_stream_writer_get_raw(GArrowIPCStreamWriter *stream_writer)
-{
-  GArrowIPCStreamWriterPrivate *priv;
-
-  priv = GARROW_IPC_STREAM_WRITER_GET_PRIVATE(stream_writer);
-  return priv->stream_writer;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-stream-writer.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-stream-writer.h b/c_glib/arrow-glib/ipc-stream-writer.h
deleted file mode 100644
index 4488204..0000000
--- a/c_glib/arrow-glib/ipc-stream-writer.h
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow-glib/array.h>
-#include <arrow-glib/record-batch.h>
-#include <arrow-glib/schema.h>
-
-#include <arrow-glib/io-output-stream.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IPC_TYPE_STREAM_WRITER           \
-  (garrow_ipc_stream_writer_get_type())
-#define GARROW_IPC_STREAM_WRITER(obj)                           \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IPC_TYPE_STREAM_WRITER,    \
-                              GArrowIPCStreamWriter))
-#define GARROW_IPC_STREAM_WRITER_CLASS(klass)                   \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
-                           GARROW_IPC_TYPE_STREAM_WRITER,       \
-                           GArrowIPCStreamWriterClass))
-#define GARROW_IPC_IS_STREAM_WRITER(obj)                        \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IPC_TYPE_STREAM_WRITER))
-#define GARROW_IPC_IS_STREAM_WRITER_CLASS(klass)                \
-  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
-                           GARROW_IPC_TYPE_STREAM_WRITER))
-#define GARROW_IPC_STREAM_WRITER_GET_CLASS(obj)                 \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_IPC_TYPE_STREAM_WRITER,     \
-                             GArrowIPCStreamWriterClass))
-
-typedef struct _GArrowIPCStreamWriter         GArrowIPCStreamWriter;
-typedef struct _GArrowIPCStreamWriterClass    GArrowIPCStreamWriterClass;
-
-/**
- * GArrowIPCStreamWriter:
- *
- * It wraps `arrow::ipc::StreamWriter`.
- */
-struct _GArrowIPCStreamWriter
-{
-  /*< private >*/
-  GObject parent_instance;
-};
-
-struct _GArrowIPCStreamWriterClass
-{
-  GObjectClass parent_class;
-};
-
-GType garrow_ipc_stream_writer_get_type(void) G_GNUC_CONST;
-
-GArrowIPCStreamWriter *garrow_ipc_stream_writer_open(GArrowIOOutputStream *sink,
-                                                     GArrowSchema *schema,
-                                                     GError **error);
-
-gboolean garrow_ipc_stream_writer_write_record_batch(GArrowIPCStreamWriter *stream_writer,
-                                                     GArrowRecordBatch *record_batch,
-                                                     GError **error);
-gboolean garrow_ipc_stream_writer_close(GArrowIPCStreamWriter *stream_writer,
-                                        GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-stream-writer.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-stream-writer.hpp b/c_glib/arrow-glib/ipc-stream-writer.hpp
deleted file mode 100644
index 9d09740..0000000
--- a/c_glib/arrow-glib/ipc-stream-writer.hpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/api.h>
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/ipc-stream-writer.h>
-
-GArrowIPCStreamWriter *garrow_ipc_stream_writer_new_raw(std::shared_ptr<arrow::ipc::StreamWriter> *arrow_stream_writer);
-std::shared_ptr<arrow::ipc::StreamWriter> garrow_ipc_stream_writer_get_raw(GArrowIPCStreamWriter *stream_writer);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/memory-mapped-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/memory-mapped-file.cpp b/c_glib/arrow-glib/memory-mapped-file.cpp
new file mode 100644
index 0000000..a3e1d0c
--- /dev/null
+++ b/c_glib/arrow-glib/memory-mapped-file.cpp
@@ -0,0 +1,287 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/io/file.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/file.hpp>
+#include <arrow-glib/file-mode.hpp>
+#include <arrow-glib/input-stream.hpp>
+#include <arrow-glib/memory-mapped-file.hpp>
+#include <arrow-glib/readable.hpp>
+#include <arrow-glib/random-access-file.hpp>
+#include <arrow-glib/writeable.hpp>
+#include <arrow-glib/writeable-file.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: memory-mapped-file
+ * @short_description: Memory mapped file class
+ *
+ * #GArrowMemoryMappedFile is a class for memory mapped file. It's
+ * readable and writeable. It supports zero copy.
+ */
+
+typedef struct GArrowMemoryMappedFilePrivate_ {
+  std::shared_ptr<arrow::io::MemoryMappedFile> memory_mapped_file;
+} GArrowMemoryMappedFilePrivate;
+
+enum {
+  PROP_0,
+  PROP_MEMORY_MAPPED_FILE
+};
+
+static std::shared_ptr<arrow::io::FileInterface>
+garrow_memory_mapped_file_get_raw_file_interface(GArrowFile *file)
+{
+  auto memory_mapped_file = GARROW_MEMORY_MAPPED_FILE(file);
+  auto arrow_memory_mapped_file =
+    garrow_memory_mapped_file_get_raw(memory_mapped_file);
+  return arrow_memory_mapped_file;
+}
+
+static void
+garrow_file_interface_init(GArrowFileInterface *iface)
+{
+  iface->get_raw = garrow_memory_mapped_file_get_raw_file_interface;
+}
+
+static std::shared_ptr<arrow::io::Readable>
+garrow_memory_mapped_file_get_raw_readable_interface(GArrowReadable *readable)
+{
+  auto memory_mapped_file = GARROW_MEMORY_MAPPED_FILE(readable);
+  auto arrow_memory_mapped_file =
+    garrow_memory_mapped_file_get_raw(memory_mapped_file);
+  return arrow_memory_mapped_file;
+}
+
+static void
+garrow_readable_interface_init(GArrowReadableInterface *iface)
+{
+  iface->get_raw = garrow_memory_mapped_file_get_raw_readable_interface;
+}
+
+static std::shared_ptr<arrow::io::InputStream>
+garrow_memory_mapped_file_get_raw_input_stream_interface(GArrowInputStream *input_stream)
+{
+  auto memory_mapped_file = GARROW_MEMORY_MAPPED_FILE(input_stream);
+  auto arrow_memory_mapped_file =
+    garrow_memory_mapped_file_get_raw(memory_mapped_file);
+  return arrow_memory_mapped_file;
+}
+
+static void
+garrow_input_stream_interface_init(GArrowInputStreamInterface *iface)
+{
+  iface->get_raw = garrow_memory_mapped_file_get_raw_input_stream_interface;
+}
+
+static std::shared_ptr<arrow::io::RandomAccessFile>
+garrow_memory_mapped_file_get_raw_random_access_file_interface(GArrowRandomAccessFile *file)
+{
+  auto memory_mapped_file = GARROW_MEMORY_MAPPED_FILE(file);
+  auto arrow_memory_mapped_file =
+    garrow_memory_mapped_file_get_raw(memory_mapped_file);
+  return arrow_memory_mapped_file;
+}
+
+static void
+garrow_random_access_file_interface_init(GArrowRandomAccessFileInterface *iface)
+{
+  iface->get_raw = garrow_memory_mapped_file_get_raw_random_access_file_interface;
+}
+
+static std::shared_ptr<arrow::io::Writeable>
+garrow_memory_mapped_file_get_raw_writeable_interface(GArrowWriteable *writeable)
+{
+  auto memory_mapped_file = GARROW_MEMORY_MAPPED_FILE(writeable);
+  auto arrow_memory_mapped_file =
+    garrow_memory_mapped_file_get_raw(memory_mapped_file);
+  return arrow_memory_mapped_file;
+}
+
+static void
+garrow_writeable_interface_init(GArrowWriteableInterface *iface)
+{
+  iface->get_raw = garrow_memory_mapped_file_get_raw_writeable_interface;
+}
+
+static std::shared_ptr<arrow::io::WriteableFile>
+garrow_memory_mapped_file_get_raw_writeable_file_interface(GArrowWriteableFile *file)
+{
+  auto memory_mapped_file = GARROW_MEMORY_MAPPED_FILE(file);
+  auto arrow_memory_mapped_file =
+    garrow_memory_mapped_file_get_raw(memory_mapped_file);
+  return arrow_memory_mapped_file;
+}
+
+static void
+garrow_writeable_file_interface_init(GArrowWriteableFileInterface *iface)
+{
+  iface->get_raw = garrow_memory_mapped_file_get_raw_writeable_file_interface;
+}
+
+G_DEFINE_TYPE_WITH_CODE(GArrowMemoryMappedFile,
+                        garrow_memory_mapped_file,
+                        G_TYPE_OBJECT,
+                        G_ADD_PRIVATE(GArrowMemoryMappedFile)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_FILE,
+                                              garrow_file_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_READABLE,
+                                              garrow_readable_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_INPUT_STREAM,
+                                              garrow_input_stream_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_RANDOM_ACCESS_FILE,
+                                              garrow_random_access_file_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_WRITEABLE,
+                                              garrow_writeable_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_WRITEABLE_FILE,
+                                              garrow_writeable_file_interface_init));
+
+#define GARROW_MEMORY_MAPPED_FILE_GET_PRIVATE(obj)                   \
+  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                                   \
+                               GARROW_TYPE_MEMORY_MAPPED_FILE,       \
+                               GArrowMemoryMappedFilePrivate))
+
+static void
+garrow_memory_mapped_file_finalize(GObject *object)
+{
+  GArrowMemoryMappedFilePrivate *priv;
+
+  priv = GARROW_MEMORY_MAPPED_FILE_GET_PRIVATE(object);
+
+  priv->memory_mapped_file = nullptr;
+
+  G_OBJECT_CLASS(garrow_memory_mapped_file_parent_class)->finalize(object);
+}
+
+static void
+garrow_memory_mapped_file_set_property(GObject *object,
+                                          guint prop_id,
+                                          const GValue *value,
+                                          GParamSpec *pspec)
+{
+  GArrowMemoryMappedFilePrivate *priv;
+
+  priv = GARROW_MEMORY_MAPPED_FILE_GET_PRIVATE(object);
+
+  switch (prop_id) {
+  case PROP_MEMORY_MAPPED_FILE:
+    priv->memory_mapped_file =
+      *static_cast<std::shared_ptr<arrow::io::MemoryMappedFile> *>(g_value_get_pointer(value));
+    break;
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_memory_mapped_file_get_property(GObject *object,
+                                          guint prop_id,
+                                          GValue *value,
+                                          GParamSpec *pspec)
+{
+  switch (prop_id) {
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_memory_mapped_file_init(GArrowMemoryMappedFile *object)
+{
+}
+
+static void
+garrow_memory_mapped_file_class_init(GArrowMemoryMappedFileClass *klass)
+{
+  GObjectClass *gobject_class;
+  GParamSpec *spec;
+
+  gobject_class = G_OBJECT_CLASS(klass);
+
+  gobject_class->finalize     = garrow_memory_mapped_file_finalize;
+  gobject_class->set_property = garrow_memory_mapped_file_set_property;
+  gobject_class->get_property = garrow_memory_mapped_file_get_property;
+
+  spec = g_param_spec_pointer("memory-mapped-file",
+                              "io::MemoryMappedFile",
+                              "The raw std::shared<arrow::io::MemoryMappedFile> *",
+                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
+                                                       G_PARAM_CONSTRUCT_ONLY));
+  g_object_class_install_property(gobject_class, PROP_MEMORY_MAPPED_FILE, spec);
+}
+
+/**
+ * garrow_memory_mapped_file_open:
+ * @path: The path of the memory mapped file.
+ * @mode: The mode of the memory mapped file.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full): A newly opened
+ *   #GArrowMemoryMappedFile or %NULL on error.
+ */
+GArrowMemoryMappedFile *
+garrow_memory_mapped_file_open(const gchar *path,
+                                  GArrowFileMode mode,
+                                  GError **error)
+{
+  std::shared_ptr<arrow::io::MemoryMappedFile> arrow_memory_mapped_file;
+  auto status =
+    arrow::io::MemoryMappedFile::Open(std::string(path),
+                                      garrow_file_mode_to_raw(mode),
+                                      &arrow_memory_mapped_file);
+  if (status.ok()) {
+    return garrow_memory_mapped_file_new_raw(&arrow_memory_mapped_file);
+  } else {
+    std::string context("[io][memory-mapped-file][open]: <");
+    context += path;
+    context += ">";
+    garrow_error_set(error, status, context.c_str());
+    return NULL;
+  }
+}
+
+G_END_DECLS
+
+GArrowMemoryMappedFile *
+garrow_memory_mapped_file_new_raw(std::shared_ptr<arrow::io::MemoryMappedFile> *arrow_memory_mapped_file)
+{
+  auto memory_mapped_file =
+    GARROW_MEMORY_MAPPED_FILE(g_object_new(GARROW_TYPE_MEMORY_MAPPED_FILE,
+                                              "memory-mapped-file", arrow_memory_mapped_file,
+                                              NULL));
+  return memory_mapped_file;
+}
+
+std::shared_ptr<arrow::io::MemoryMappedFile>
+garrow_memory_mapped_file_get_raw(GArrowMemoryMappedFile *memory_mapped_file)
+{
+  GArrowMemoryMappedFilePrivate *priv;
+
+  priv = GARROW_MEMORY_MAPPED_FILE_GET_PRIVATE(memory_mapped_file);
+  return priv->memory_mapped_file;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/memory-mapped-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/memory-mapped-file.h b/c_glib/arrow-glib/memory-mapped-file.h
new file mode 100644
index 0000000..40b8de0
--- /dev/null
+++ b/c_glib/arrow-glib/memory-mapped-file.h
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow-glib/file-mode.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_MEMORY_MAPPED_FILE       \
+  (garrow_memory_mapped_file_get_type())
+#define GARROW_MEMORY_MAPPED_FILE(obj)                               \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                                    \
+                              GARROW_TYPE_MEMORY_MAPPED_FILE,        \
+                              GArrowMemoryMappedFile))
+#define GARROW_MEMORY_MAPPED_FILE_CLASS(klass)               \
+  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
+                           GARROW_TYPE_MEMORY_MAPPED_FILE,   \
+                           GArrowMemoryMappedFileClass))
+#define GARROW_IS_MEMORY_MAPPED_FILE(obj)                            \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                                    \
+                              GARROW_TYPE_MEMORY_MAPPED_FILE))
+#define GARROW_IS_MEMORY_MAPPED_FILE_CLASS(klass)            \
+  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
+                           GARROW_TYPE_MEMORY_MAPPED_FILE))
+#define GARROW_MEMORY_MAPPED_FILE_GET_CLASS(obj)             \
+  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
+                             GARROW_TYPE_MEMORY_MAPPED_FILE, \
+                             GArrowMemoryMappedFileClass))
+
+typedef struct _GArrowMemoryMappedFile         GArrowMemoryMappedFile;
+typedef struct _GArrowMemoryMappedFileClass    GArrowMemoryMappedFileClass;
+
+/**
+ * GArrowMemoryMappedFile:
+ *
+ * It wraps `arrow::io::MemoryMappedFile`.
+ */
+struct _GArrowMemoryMappedFile
+{
+  /*< private >*/
+  GObject parent_instance;
+};
+
+struct _GArrowMemoryMappedFileClass
+{
+  GObjectClass parent_class;
+};
+
+GType garrow_memory_mapped_file_get_type(void) G_GNUC_CONST;
+
+GArrowMemoryMappedFile *garrow_memory_mapped_file_open(const gchar *path,
+                                                            GArrowFileMode mode,
+                                                            GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/memory-mapped-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/memory-mapped-file.hpp b/c_glib/arrow-glib/memory-mapped-file.hpp
new file mode 100644
index 0000000..522e43d
--- /dev/null
+++ b/c_glib/arrow-glib/memory-mapped-file.hpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/api.h>
+#include <arrow/io/file.h>
+
+#include <arrow-glib/memory-mapped-file.h>
+
+GArrowMemoryMappedFile *garrow_memory_mapped_file_new_raw(std::shared_ptr<arrow::io::MemoryMappedFile> *arrow_memory_mapped_file);
+std::shared_ptr<arrow::io::MemoryMappedFile> garrow_memory_mapped_file_get_raw(GArrowMemoryMappedFile *memory_mapped_file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/metadata-version.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/metadata-version.cpp b/c_glib/arrow-glib/metadata-version.cpp
new file mode 100644
index 0000000..ee458eb
--- /dev/null
+++ b/c_glib/arrow-glib/metadata-version.cpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow-glib/metadata-version.hpp>
+
+/**
+ * SECTION: metadata-version
+ * @title: GArrowMetadataVersion
+ * @short_description: Metadata version mapgging between Arrow and arrow-glib
+ *
+ * #GArrowMetadataVersion provides metadata versions corresponding
+ * to `arrow::ipc::MetadataVersion` values.
+ */
+
+GArrowMetadataVersion
+garrow_metadata_version_from_raw(arrow::ipc::MetadataVersion version)
+{
+  switch (version) {
+  case arrow::ipc::MetadataVersion::V1:
+    return GARROW_METADATA_VERSION_V1;
+  case arrow::ipc::MetadataVersion::V2:
+    return GARROW_METADATA_VERSION_V2;
+  case arrow::ipc::MetadataVersion::V3:
+    return GARROW_METADATA_VERSION_V3;
+  default:
+    return GARROW_METADATA_VERSION_V3;
+  }
+}
+
+arrow::ipc::MetadataVersion
+garrow_metadata_version_to_raw(GArrowMetadataVersion version)
+{
+  switch (version) {
+  case GARROW_METADATA_VERSION_V1:
+    return arrow::ipc::MetadataVersion::V1;
+  case GARROW_METADATA_VERSION_V2:
+    return arrow::ipc::MetadataVersion::V2;
+  case GARROW_METADATA_VERSION_V3:
+    return arrow::ipc::MetadataVersion::V3;
+  default:
+    return arrow::ipc::MetadataVersion::V3;
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/metadata-version.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/metadata-version.h b/c_glib/arrow-glib/metadata-version.h
new file mode 100644
index 0000000..d902a39
--- /dev/null
+++ b/c_glib/arrow-glib/metadata-version.h
@@ -0,0 +1,41 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+/**
+ * GArrowMetadataVersion:
+ * @GARROW_METADATA_VERSION_V1: Version 1.
+ * @GARROW_METADATA_VERSION_V2: Version 2.
+ * @GARROW_METADATA_VERSION_V3: Version 3.
+ *
+ * They are corresponding to `arrow::ipc::MetadataVersion::type`
+ * values.
+ */
+typedef enum {
+  GARROW_METADATA_VERSION_V1,
+  GARROW_METADATA_VERSION_V2,
+  GARROW_METADATA_VERSION_V3
+} GArrowMetadataVersion;
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/metadata-version.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/metadata-version.hpp b/c_glib/arrow-glib/metadata-version.hpp
new file mode 100644
index 0000000..7b3865e
--- /dev/null
+++ b/c_glib/arrow-glib/metadata-version.hpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/metadata-version.h>
+
+GArrowMetadataVersion garrow_metadata_version_from_raw(arrow::ipc::MetadataVersion version);
+arrow::ipc::MetadataVersion garrow_metadata_version_to_raw(GArrowMetadataVersion version);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/output-stream.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/output-stream.cpp b/c_glib/arrow-glib/output-stream.cpp
new file mode 100644
index 0000000..bbc29b7
--- /dev/null
+++ b/c_glib/arrow-glib/output-stream.cpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/output-stream.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: output-stream
+ * @title: GArrowOutputStream
+ * @short_description: Stream output interface
+ *
+ * #GArrowOutputStream is an interface for stream output. Stream
+ * output is file based and writeable
+ */
+
+G_DEFINE_INTERFACE(GArrowOutputStream,
+                   garrow_output_stream,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_output_stream_default_init (GArrowOutputStreamInterface *iface)
+{
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::OutputStream>
+garrow_output_stream_get_raw(GArrowOutputStream *output_stream)
+{
+  auto *iface = GARROW_OUTPUT_STREAM_GET_IFACE(output_stream);
+  return iface->get_raw(output_stream);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/output-stream.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/output-stream.h b/c_glib/arrow-glib/output-stream.h
new file mode 100644
index 0000000..3481072
--- /dev/null
+++ b/c_glib/arrow-glib/output-stream.h
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_OUTPUT_STREAM            \
+  (garrow_output_stream_get_type())
+#define GARROW_OUTPUT_STREAM(obj)                            \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_OUTPUT_STREAM,     \
+                              GArrowOutputStream))
+#define GARROW_IS_OUTPUT_STREAM(obj)                         \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_OUTPUT_STREAM))
+#define GARROW_OUTPUT_STREAM_GET_IFACE(obj)                          \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                                 \
+                                 GARROW_TYPE_OUTPUT_STREAM,          \
+                                 GArrowOutputStreamInterface))
+
+typedef struct _GArrowOutputStream          GArrowOutputStream;
+typedef struct _GArrowOutputStreamInterface GArrowOutputStreamInterface;
+
+GType garrow_output_stream_get_type(void) G_GNUC_CONST;
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/output-stream.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/output-stream.hpp b/c_glib/arrow-glib/output-stream.hpp
new file mode 100644
index 0000000..635da10
--- /dev/null
+++ b/c_glib/arrow-glib/output-stream.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/output-stream.h>
+
+/**
+ * GArrowOutputStreamInterface:
+ *
+ * It wraps `arrow::io::OutputStream`.
+ */
+struct _GArrowOutputStreamInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::OutputStream> (*get_raw)(GArrowOutputStream *file);
+};
+
+std::shared_ptr<arrow::io::OutputStream> garrow_output_stream_get_raw(GArrowOutputStream *output_stream);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/random-access-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/random-access-file.cpp b/c_glib/arrow-glib/random-access-file.cpp
new file mode 100644
index 0000000..71f315e
--- /dev/null
+++ b/c_glib/arrow-glib/random-access-file.cpp
@@ -0,0 +1,128 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/random-access-file.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: random-access-file
+ * @title: GArrowRandomAccessFile
+ * @short_description: File input interface
+ *
+ * #GArrowRandomAccessFile is an interface for file input.
+ */
+
+G_DEFINE_INTERFACE(GArrowRandomAccessFile,
+                   garrow_random_access_file,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_random_access_file_default_init (GArrowRandomAccessFileInterface *iface)
+{
+}
+
+/**
+ * garrow_random_access_file_get_size:
+ * @file: A #GArrowRandomAccessFile.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: The size of the file.
+ */
+guint64
+garrow_random_access_file_get_size(GArrowRandomAccessFile *file,
+                                 GError **error)
+{
+  auto *iface = GARROW_RANDOM_ACCESS_FILE_GET_IFACE(file);
+  auto arrow_random_access_file = iface->get_raw(file);
+  int64_t size;
+
+  auto status = arrow_random_access_file->GetSize(&size);
+  if (status.ok()) {
+    return size;
+  } else {
+    garrow_error_set(error, status, "[io][random-access-file][get-size]");
+    return 0;
+  }
+}
+
+/**
+ * garrow_random_access_file_get_support_zero_copy:
+ * @file: A #GArrowRandomAccessFile.
+ *
+ * Returns: Whether zero copy read is supported or not.
+ */
+gboolean
+garrow_random_access_file_get_support_zero_copy(GArrowRandomAccessFile *file)
+{
+  auto *iface = GARROW_RANDOM_ACCESS_FILE_GET_IFACE(file);
+  auto arrow_random_access_file = iface->get_raw(file);
+
+  return arrow_random_access_file->supports_zero_copy();
+}
+
+/**
+ * garrow_random_access_file_read_at:
+ * @file: A #GArrowRandomAccessFile.
+ * @position: The read start position.
+ * @n_bytes: The number of bytes to be read.
+ * @n_read_bytes: (out): The read number of bytes.
+ * @buffer: (array length=n_bytes): The buffer to be read data.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_random_access_file_read_at(GArrowRandomAccessFile *file,
+                                     gint64 position,
+                                     gint64 n_bytes,
+                                     gint64 *n_read_bytes,
+                                     guint8 *buffer,
+                                     GError **error)
+{
+  const auto arrow_random_access_file =
+    garrow_random_access_file_get_raw(file);
+
+  auto status = arrow_random_access_file->ReadAt(position,
+                                                 n_bytes,
+                                                 n_read_bytes,
+                                                 buffer);
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[io][random-access-file][read-at]");
+    return FALSE;
+  }
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::RandomAccessFile>
+garrow_random_access_file_get_raw(GArrowRandomAccessFile *random_access_file)
+{
+  auto *iface = GARROW_RANDOM_ACCESS_FILE_GET_IFACE(random_access_file);
+  return iface->get_raw(random_access_file);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/random-access-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/random-access-file.h b/c_glib/arrow-glib/random-access-file.h
new file mode 100644
index 0000000..8a7f6b4
--- /dev/null
+++ b/c_glib/arrow-glib/random-access-file.h
@@ -0,0 +1,55 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_RANDOM_ACCESS_FILE       \
+  (garrow_random_access_file_get_type())
+#define GARROW_RANDOM_ACCESS_FILE(obj)                            \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                                 \
+                              GARROW_TYPE_RANDOM_ACCESS_FILE,     \
+                              GArrowRandomAccessFile))
+#define GARROW_IS_RANDOM_ACCESS_FILE(obj)                            \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                                    \
+                              GARROW_TYPE_RANDOM_ACCESS_FILE))
+#define GARROW_RANDOM_ACCESS_FILE_GET_IFACE(obj)                     \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                                 \
+                                 GARROW_TYPE_RANDOM_ACCESS_FILE,     \
+                                 GArrowRandomAccessFileInterface))
+
+typedef struct _GArrowRandomAccessFile          GArrowRandomAccessFile;
+typedef struct _GArrowRandomAccessFileInterface GArrowRandomAccessFileInterface;
+
+GType garrow_random_access_file_get_type(void) G_GNUC_CONST;
+
+guint64 garrow_random_access_file_get_size(GArrowRandomAccessFile *file,
+                                              GError **error);
+gboolean garrow_random_access_file_get_support_zero_copy(GArrowRandomAccessFile *file);
+gboolean garrow_random_access_file_read_at(GArrowRandomAccessFile *file,
+                                              gint64 position,
+                                              gint64 n_bytes,
+                                              gint64 *n_read_bytes,
+                                              guint8 *buffer,
+                                              GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/random-access-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/random-access-file.hpp b/c_glib/arrow-glib/random-access-file.hpp
new file mode 100644
index 0000000..6d6fed7
--- /dev/null
+++ b/c_glib/arrow-glib/random-access-file.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/random-access-file.h>
+
+/**
+ * GArrowRandomAccessFileInterface:
+ *
+ * It wraps `arrow::io::RandomAccessFile`.
+ */
+struct _GArrowRandomAccessFileInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::RandomAccessFile> (*get_raw)(GArrowRandomAccessFile *file);
+};
+
+std::shared_ptr<arrow::io::RandomAccessFile> garrow_random_access_file_get_raw(GArrowRandomAccessFile *random_access_file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/readable.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/readable.cpp b/c_glib/arrow-glib/readable.cpp
new file mode 100644
index 0000000..b8c0cd9
--- /dev/null
+++ b/c_glib/arrow-glib/readable.cpp
@@ -0,0 +1,84 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/readable.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: readable
+ * @title: GArrowReadable
+ * @short_description: Input interface
+ *
+ * #GArrowReadable is an interface for input. Input must be
+ * readable.
+ */
+
+G_DEFINE_INTERFACE(GArrowReadable,
+                   garrow_readable,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_readable_default_init (GArrowReadableInterface *iface)
+{
+}
+
+/**
+ * garrow_readable_read:
+ * @readable: A #GArrowReadable.
+ * @n_bytes: The number of bytes to be read.
+ * @n_read_bytes: (out): The read number of bytes.
+ * @buffer: (array length=n_bytes): The buffer to be read data.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_readable_read(GArrowReadable *readable,
+                        gint64 n_bytes,
+                        gint64 *n_read_bytes,
+                        guint8 *buffer,
+                        GError **error)
+{
+  const auto arrow_readable = garrow_readable_get_raw(readable);
+
+  auto status = arrow_readable->Read(n_bytes, n_read_bytes, buffer);
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[io][readable][read]");
+    return FALSE;
+  }
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::Readable>
+garrow_readable_get_raw(GArrowReadable *readable)
+{
+  auto *iface = GARROW_READABLE_GET_IFACE(readable);
+  return iface->get_raw(readable);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/readable.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/readable.h b/c_glib/arrow-glib/readable.h
new file mode 100644
index 0000000..bde4b01
--- /dev/null
+++ b/c_glib/arrow-glib/readable.h
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_READABLE                 \
+  (garrow_readable_get_type())
+#define GARROW_READABLE(obj)                                 \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_READABLE,          \
+                              GArrowReadable))
+#define GARROW_IS_READABLE(obj)                      \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                    \
+                              GARROW_TYPE_READABLE))
+#define GARROW_READABLE_GET_IFACE(obj)                       \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
+                                 GARROW_TYPE_READABLE,       \
+                                 GArrowReadableInterface))
+
+typedef struct _GArrowReadable          GArrowReadable;
+typedef struct _GArrowReadableInterface GArrowReadableInterface;
+
+GType garrow_readable_get_type(void) G_GNUC_CONST;
+
+gboolean garrow_readable_read(GArrowReadable *readable,
+                                 gint64 n_bytes,
+                                 gint64 *n_read_bytes,
+                                 guint8 *buffer,
+                                 GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/readable.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/readable.hpp b/c_glib/arrow-glib/readable.hpp
new file mode 100644
index 0000000..c241c77
--- /dev/null
+++ b/c_glib/arrow-glib/readable.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/readable.h>
+
+/**
+ * GArrowReadableInterface:
+ *
+ * It wraps `arrow::io::Readable`.
+ */
+struct _GArrowReadableInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::Readable> (*get_raw)(GArrowReadable *file);
+};
+
+std::shared_ptr<arrow::io::Readable> garrow_readable_get_raw(GArrowReadable *readable);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/stream-reader.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/stream-reader.cpp b/c_glib/arrow-glib/stream-reader.cpp
new file mode 100644
index 0000000..c4ccebe
--- /dev/null
+++ b/c_glib/arrow-glib/stream-reader.cpp
@@ -0,0 +1,221 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/record-batch.hpp>
+#include <arrow-glib/schema.hpp>
+
+#include <arrow-glib/input-stream.hpp>
+
+#include <arrow-glib/metadata-version.hpp>
+#include <arrow-glib/stream-reader.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: stream-reader
+ * @short_description: Stream reader class
+ *
+ * #GArrowStreamReader is a class for receiving data by stream
+ * based IPC.
+ */
+
+typedef struct GArrowStreamReaderPrivate_ {
+  std::shared_ptr<arrow::ipc::StreamReader> stream_reader;
+} GArrowStreamReaderPrivate;
+
+enum {
+  PROP_0,
+  PROP_STREAM_READER
+};
+
+G_DEFINE_TYPE_WITH_PRIVATE(GArrowStreamReader,
+                           garrow_stream_reader,
+                           G_TYPE_OBJECT);
+
+#define GARROW_STREAM_READER_GET_PRIVATE(obj)               \
+  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                           \
+                               GARROW_TYPE_STREAM_READER,   \
+                               GArrowStreamReaderPrivate))
+
+static void
+garrow_stream_reader_finalize(GObject *object)
+{
+  GArrowStreamReaderPrivate *priv;
+
+  priv = GARROW_STREAM_READER_GET_PRIVATE(object);
+
+  priv->stream_reader = nullptr;
+
+  G_OBJECT_CLASS(garrow_stream_reader_parent_class)->finalize(object);
+}
+
+static void
+garrow_stream_reader_set_property(GObject *object,
+                                    guint prop_id,
+                                    const GValue *value,
+                                    GParamSpec *pspec)
+{
+  GArrowStreamReaderPrivate *priv;
+
+  priv = GARROW_STREAM_READER_GET_PRIVATE(object);
+
+  switch (prop_id) {
+  case PROP_STREAM_READER:
+    priv->stream_reader =
+      *static_cast<std::shared_ptr<arrow::ipc::StreamReader> *>(g_value_get_pointer(value));
+    break;
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_stream_reader_get_property(GObject *object,
+                                          guint prop_id,
+                                          GValue *value,
+                                          GParamSpec *pspec)
+{
+  switch (prop_id) {
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_stream_reader_init(GArrowStreamReader *object)
+{
+}
+
+static void
+garrow_stream_reader_class_init(GArrowStreamReaderClass *klass)
+{
+  GObjectClass *gobject_class;
+  GParamSpec *spec;
+
+  gobject_class = G_OBJECT_CLASS(klass);
+
+  gobject_class->finalize     = garrow_stream_reader_finalize;
+  gobject_class->set_property = garrow_stream_reader_set_property;
+  gobject_class->get_property = garrow_stream_reader_get_property;
+
+  spec = g_param_spec_pointer("stream-reader",
+                              "ipc::StreamReader",
+                              "The raw std::shared<arrow::ipc::StreamReader> *",
+                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
+                                                       G_PARAM_CONSTRUCT_ONLY));
+  g_object_class_install_property(gobject_class, PROP_STREAM_READER, spec);
+}
+
+/**
+ * garrow_stream_reader_open:
+ * @stream: The stream to be read.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full): A newly opened
+ *   #GArrowStreamReader or %NULL on error.
+ */
+GArrowStreamReader *
+garrow_stream_reader_open(GArrowInputStream *stream,
+                              GError **error)
+{
+  std::shared_ptr<arrow::ipc::StreamReader> arrow_stream_reader;
+  auto status =
+    arrow::ipc::StreamReader::Open(garrow_input_stream_get_raw(stream),
+                                   &arrow_stream_reader);
+  if (status.ok()) {
+    return garrow_stream_reader_new_raw(&arrow_stream_reader);
+  } else {
+    garrow_error_set(error, status, "[ipc][stream-reader][open]");
+    return NULL;
+  }
+}
+
+/**
+ * garrow_stream_reader_get_schema:
+ * @stream_reader: A #GArrowStreamReader.
+ *
+ * Returns: (transfer full): The schema in the stream.
+ */
+GArrowSchema *
+garrow_stream_reader_get_schema(GArrowStreamReader *stream_reader)
+{
+  auto arrow_stream_reader =
+    garrow_stream_reader_get_raw(stream_reader);
+  auto arrow_schema = arrow_stream_reader->schema();
+  return garrow_schema_new_raw(&arrow_schema);
+}
+
+/**
+ * garrow_stream_reader_get_next_record_batch:
+ * @stream_reader: A #GArrowStreamReader.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full):
+ *   The next record batch in the stream or %NULL on end of stream.
+ */
+GArrowRecordBatch *
+garrow_stream_reader_get_next_record_batch(GArrowStreamReader *stream_reader,
+                                               GError **error)
+{
+  auto arrow_stream_reader =
+    garrow_stream_reader_get_raw(stream_reader);
+  std::shared_ptr<arrow::RecordBatch> arrow_record_batch;
+  auto status = arrow_stream_reader->GetNextRecordBatch(&arrow_record_batch);
+
+  if (status.ok()) {
+    if (arrow_record_batch == nullptr) {
+      return NULL;
+    } else {
+      return garrow_record_batch_new_raw(&arrow_record_batch);
+    }
+  } else {
+    garrow_error_set(error, status, "[ipc][stream-reader][get-next-record-batch]");
+    return NULL;
+  }
+}
+
+G_END_DECLS
+
+GArrowStreamReader *
+garrow_stream_reader_new_raw(std::shared_ptr<arrow::ipc::StreamReader> *arrow_stream_reader)
+{
+  auto stream_reader =
+    GARROW_STREAM_READER(g_object_new(GARROW_TYPE_STREAM_READER,
+                                          "stream-reader", arrow_stream_reader,
+                                          NULL));
+  return stream_reader;
+}
+
+std::shared_ptr<arrow::ipc::StreamReader>
+garrow_stream_reader_get_raw(GArrowStreamReader *stream_reader)
+{
+  GArrowStreamReaderPrivate *priv;
+
+  priv = GARROW_STREAM_READER_GET_PRIVATE(stream_reader);
+  return priv->stream_reader;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/stream-reader.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/stream-reader.h b/c_glib/arrow-glib/stream-reader.h
new file mode 100644
index 0000000..16a7f57
--- /dev/null
+++ b/c_glib/arrow-glib/stream-reader.h
@@ -0,0 +1,80 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow-glib/record-batch.h>
+#include <arrow-glib/schema.h>
+
+#include <arrow-glib/input-stream.h>
+
+#include <arrow-glib/metadata-version.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_STREAM_READER           \
+  (garrow_stream_reader_get_type())
+#define GARROW_STREAM_READER(obj)                           \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_STREAM_READER,    \
+                              GArrowStreamReader))
+#define GARROW_STREAM_READER_CLASS(klass)                   \
+  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
+                           GARROW_TYPE_STREAM_READER,       \
+                           GArrowStreamReaderClass))
+#define GARROW_IS_STREAM_READER(obj)                        \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_STREAM_READER))
+#define GARROW_IS_STREAM_READER_CLASS(klass)                \
+  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
+                           GARROW_TYPE_STREAM_READER))
+#define GARROW_STREAM_READER_GET_CLASS(obj)                 \
+  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
+                             GARROW_TYPE_STREAM_READER,     \
+                             GArrowStreamReaderClass))
+
+typedef struct _GArrowStreamReader         GArrowStreamReader;
+typedef struct _GArrowStreamReaderClass    GArrowStreamReaderClass;
+
+/**
+ * GArrowStreamReader:
+ *
+ * It wraps `arrow::ipc::StreamReader`.
+ */
+struct _GArrowStreamReader
+{
+  /*< private >*/
+  GObject parent_instance;
+};
+
+struct _GArrowStreamReaderClass
+{
+  GObjectClass parent_class;
+};
+
+GType garrow_stream_reader_get_type(void) G_GNUC_CONST;
+
+GArrowStreamReader *garrow_stream_reader_open(GArrowInputStream *stream,
+                                                     GError **error);
+
+GArrowSchema *garrow_stream_reader_get_schema(GArrowStreamReader *stream_reader);
+GArrowRecordBatch *garrow_stream_reader_get_next_record_batch(GArrowStreamReader *stream_reader,
+                                                                  GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/stream-reader.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/stream-reader.hpp b/c_glib/arrow-glib/stream-reader.hpp
new file mode 100644
index 0000000..ca8e689
--- /dev/null
+++ b/c_glib/arrow-glib/stream-reader.hpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/api.h>
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/stream-reader.h>
+
+GArrowStreamReader *garrow_stream_reader_new_raw(std::shared_ptr<arrow::ipc::StreamReader> *arrow_stream_reader);
+std::shared_ptr<arrow::ipc::StreamReader> garrow_stream_reader_get_raw(GArrowStreamReader *stream_reader);


[3/4] arrow git commit: ARROW-810: [GLib] Remove io/ipc prefix

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file.cpp b/c_glib/arrow-glib/io-file.cpp
deleted file mode 100644
index 536ae3e..0000000
--- a/c_glib/arrow-glib/io-file.cpp
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-file.hpp>
-#include <arrow-glib/io-file-mode.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-file
- * @title: GArrowIOFile
- * @short_description: File interface
- *
- * #GArrowIOFile is an interface for file.
- */
-
-G_DEFINE_INTERFACE(GArrowIOFile,
-                   garrow_io_file,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_file_default_init (GArrowIOFileInterface *iface)
-{
-}
-
-/**
- * garrow_io_file_close:
- * @file: A #GArrowIOFile.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_io_file_close(GArrowIOFile *file,
-                     GError **error)
-{
-  auto arrow_file = garrow_io_file_get_raw(file);
-
-  auto status = arrow_file->Close();
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[io][file][close]");
-    return FALSE;
-  }
-}
-
-/**
- * garrow_io_file_tell:
- * @file: A #GArrowIOFile.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: The current offset on success, -1 if there was an error.
- */
-gint64
-garrow_io_file_tell(GArrowIOFile *file,
-                    GError **error)
-{
-  auto arrow_file = garrow_io_file_get_raw(file);
-
-  gint64 position;
-  auto status = arrow_file->Tell(&position);
-  if (status.ok()) {
-    return position;
-  } else {
-    garrow_error_set(error, status, "[io][file][tell]");
-    return -1;
-  }
-}
-
-/**
- * garrow_io_file_get_mode:
- * @file: A #GArrowIOFile.
- *
- * Returns: The mode of the file.
- */
-GArrowIOFileMode
-garrow_io_file_get_mode(GArrowIOFile *file)
-{
-  auto arrow_file = garrow_io_file_get_raw(file);
-
-  auto arrow_mode = arrow_file->mode();
-  return garrow_io_file_mode_from_raw(arrow_mode);
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::FileInterface>
-garrow_io_file_get_raw(GArrowIOFile *file)
-{
-  auto *iface = GARROW_IO_FILE_GET_IFACE(file);
-  return iface->get_raw(file);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file.h b/c_glib/arrow-glib/io-file.h
deleted file mode 100644
index 7181f6d..0000000
--- a/c_glib/arrow-glib/io-file.h
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow-glib/io-file-mode.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_FILE                     \
-  (garrow_io_file_get_type())
-#define GARROW_IO_FILE(obj)                             \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                    \
-                              GARROW_IO_TYPE_FILE,      \
-                              GArrowIOFile))
-#define GARROW_IO_IS_FILE(obj)                          \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                    \
-                              GARROW_IO_TYPE_FILE))
-#define GARROW_IO_FILE_GET_IFACE(obj)                           \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
-                                 GARROW_IO_TYPE_FILE,           \
-                                 GArrowIOFileInterface))
-
-typedef struct _GArrowIOFile          GArrowIOFile;
-typedef struct _GArrowIOFileInterface GArrowIOFileInterface;
-
-GType garrow_io_file_get_type(void) G_GNUC_CONST;
-
-gboolean garrow_io_file_close(GArrowIOFile *file,
-                              GError **error);
-gint64 garrow_io_file_tell(GArrowIOFile *file,
-                           GError **error);
-GArrowIOFileMode garrow_io_file_get_mode(GArrowIOFile *file);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file.hpp b/c_glib/arrow-glib/io-file.hpp
deleted file mode 100644
index afaca90..0000000
--- a/c_glib/arrow-glib/io-file.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-file.h>
-
-/**
- * GArrowIOFileInterface:
- *
- * It wraps `arrow::io::FileInterface`.
- */
-struct _GArrowIOFileInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::FileInterface> (*get_raw)(GArrowIOFile *file);
-};
-
-std::shared_ptr<arrow::io::FileInterface> garrow_io_file_get_raw(GArrowIOFile *file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-input-stream.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-input-stream.cpp b/c_glib/arrow-glib/io-input-stream.cpp
deleted file mode 100644
index a28b9c6..0000000
--- a/c_glib/arrow-glib/io-input-stream.cpp
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-input-stream.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-input-stream
- * @title: GArrowIOInputStream
- * @short_description: Stream input interface
- *
- * #GArrowIOInputStream is an interface for stream input. Stream input
- * is file based and readable.
- */
-
-G_DEFINE_INTERFACE(GArrowIOInputStream,
-                   garrow_io_input_stream,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_input_stream_default_init (GArrowIOInputStreamInterface *iface)
-{
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::InputStream>
-garrow_io_input_stream_get_raw(GArrowIOInputStream *input_stream)
-{
-  auto *iface = GARROW_IO_INPUT_STREAM_GET_IFACE(input_stream);
-  return iface->get_raw(input_stream);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-input-stream.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-input-stream.h b/c_glib/arrow-glib/io-input-stream.h
deleted file mode 100644
index 5790209..0000000
--- a/c_glib/arrow-glib/io-input-stream.h
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_INPUT_STREAM             \
-  (garrow_io_input_stream_get_type())
-#define GARROW_IO_INPUT_STREAM(obj)                             \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IO_TYPE_INPUT_STREAM,      \
-                              GArrowIOInputStream))
-#define GARROW_IO_IS_INPUT_STREAM(obj)                          \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IO_TYPE_INPUT_STREAM))
-#define GARROW_IO_INPUT_STREAM_GET_IFACE(obj)                   \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
-                                 GARROW_IO_TYPE_INPUT_STREAM,   \
-                                 GArrowIOInputStreamInterface))
-
-typedef struct _GArrowIOInputStream          GArrowIOInputStream;
-typedef struct _GArrowIOInputStreamInterface GArrowIOInputStreamInterface;
-
-GType garrow_io_input_stream_get_type(void) G_GNUC_CONST;
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-input-stream.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-input-stream.hpp b/c_glib/arrow-glib/io-input-stream.hpp
deleted file mode 100644
index 3b1de5d..0000000
--- a/c_glib/arrow-glib/io-input-stream.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-input-stream.h>
-
-/**
- * GArrowIOInputStreamInterface:
- *
- * It wraps `arrow::io::InputStream`.
- */
-struct _GArrowIOInputStreamInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::InputStream> (*get_raw)(GArrowIOInputStream *file);
-};
-
-std::shared_ptr<arrow::io::InputStream> garrow_io_input_stream_get_raw(GArrowIOInputStream *input_stream);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-memory-mapped-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-memory-mapped-file.cpp b/c_glib/arrow-glib/io-memory-mapped-file.cpp
deleted file mode 100644
index e2e255c..0000000
--- a/c_glib/arrow-glib/io-memory-mapped-file.cpp
+++ /dev/null
@@ -1,287 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/io/file.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-file.hpp>
-#include <arrow-glib/io-file-mode.hpp>
-#include <arrow-glib/io-input-stream.hpp>
-#include <arrow-glib/io-memory-mapped-file.hpp>
-#include <arrow-glib/io-readable.hpp>
-#include <arrow-glib/io-random-access-file.hpp>
-#include <arrow-glib/io-writeable.hpp>
-#include <arrow-glib/io-writeable-file.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-memory-mapped-file
- * @short_description: Memory mapped file class
- *
- * #GArrowIOMemoryMappedFile is a class for memory mapped file. It's
- * readable and writeable. It supports zero copy.
- */
-
-typedef struct GArrowIOMemoryMappedFilePrivate_ {
-  std::shared_ptr<arrow::io::MemoryMappedFile> memory_mapped_file;
-} GArrowIOMemoryMappedFilePrivate;
-
-enum {
-  PROP_0,
-  PROP_MEMORY_MAPPED_FILE
-};
-
-static std::shared_ptr<arrow::io::FileInterface>
-garrow_io_memory_mapped_file_get_raw_file_interface(GArrowIOFile *file)
-{
-  auto memory_mapped_file = GARROW_IO_MEMORY_MAPPED_FILE(file);
-  auto arrow_memory_mapped_file =
-    garrow_io_memory_mapped_file_get_raw(memory_mapped_file);
-  return arrow_memory_mapped_file;
-}
-
-static void
-garrow_io_file_interface_init(GArrowIOFileInterface *iface)
-{
-  iface->get_raw = garrow_io_memory_mapped_file_get_raw_file_interface;
-}
-
-static std::shared_ptr<arrow::io::Readable>
-garrow_io_memory_mapped_file_get_raw_readable_interface(GArrowIOReadable *readable)
-{
-  auto memory_mapped_file = GARROW_IO_MEMORY_MAPPED_FILE(readable);
-  auto arrow_memory_mapped_file =
-    garrow_io_memory_mapped_file_get_raw(memory_mapped_file);
-  return arrow_memory_mapped_file;
-}
-
-static void
-garrow_io_readable_interface_init(GArrowIOReadableInterface *iface)
-{
-  iface->get_raw = garrow_io_memory_mapped_file_get_raw_readable_interface;
-}
-
-static std::shared_ptr<arrow::io::InputStream>
-garrow_io_memory_mapped_file_get_raw_input_stream_interface(GArrowIOInputStream *input_stream)
-{
-  auto memory_mapped_file = GARROW_IO_MEMORY_MAPPED_FILE(input_stream);
-  auto arrow_memory_mapped_file =
-    garrow_io_memory_mapped_file_get_raw(memory_mapped_file);
-  return arrow_memory_mapped_file;
-}
-
-static void
-garrow_io_input_stream_interface_init(GArrowIOInputStreamInterface *iface)
-{
-  iface->get_raw = garrow_io_memory_mapped_file_get_raw_input_stream_interface;
-}
-
-static std::shared_ptr<arrow::io::RandomAccessFile>
-garrow_io_memory_mapped_file_get_raw_random_access_file_interface(GArrowIORandomAccessFile *file)
-{
-  auto memory_mapped_file = GARROW_IO_MEMORY_MAPPED_FILE(file);
-  auto arrow_memory_mapped_file =
-    garrow_io_memory_mapped_file_get_raw(memory_mapped_file);
-  return arrow_memory_mapped_file;
-}
-
-static void
-garrow_io_random_access_file_interface_init(GArrowIORandomAccessFileInterface *iface)
-{
-  iface->get_raw = garrow_io_memory_mapped_file_get_raw_random_access_file_interface;
-}
-
-static std::shared_ptr<arrow::io::Writeable>
-garrow_io_memory_mapped_file_get_raw_writeable_interface(GArrowIOWriteable *writeable)
-{
-  auto memory_mapped_file = GARROW_IO_MEMORY_MAPPED_FILE(writeable);
-  auto arrow_memory_mapped_file =
-    garrow_io_memory_mapped_file_get_raw(memory_mapped_file);
-  return arrow_memory_mapped_file;
-}
-
-static void
-garrow_io_writeable_interface_init(GArrowIOWriteableInterface *iface)
-{
-  iface->get_raw = garrow_io_memory_mapped_file_get_raw_writeable_interface;
-}
-
-static std::shared_ptr<arrow::io::WriteableFile>
-garrow_io_memory_mapped_file_get_raw_writeable_file_interface(GArrowIOWriteableFile *file)
-{
-  auto memory_mapped_file = GARROW_IO_MEMORY_MAPPED_FILE(file);
-  auto arrow_memory_mapped_file =
-    garrow_io_memory_mapped_file_get_raw(memory_mapped_file);
-  return arrow_memory_mapped_file;
-}
-
-static void
-garrow_io_writeable_file_interface_init(GArrowIOWriteableFileInterface *iface)
-{
-  iface->get_raw = garrow_io_memory_mapped_file_get_raw_writeable_file_interface;
-}
-
-G_DEFINE_TYPE_WITH_CODE(GArrowIOMemoryMappedFile,
-                        garrow_io_memory_mapped_file,
-                        G_TYPE_OBJECT,
-                        G_ADD_PRIVATE(GArrowIOMemoryMappedFile)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_FILE,
-                                              garrow_io_file_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_READABLE,
-                                              garrow_io_readable_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_INPUT_STREAM,
-                                              garrow_io_input_stream_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_RANDOM_ACCESS_FILE,
-                                              garrow_io_random_access_file_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_WRITEABLE,
-                                              garrow_io_writeable_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_WRITEABLE_FILE,
-                                              garrow_io_writeable_file_interface_init));
-
-#define GARROW_IO_MEMORY_MAPPED_FILE_GET_PRIVATE(obj)                   \
-  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                                   \
-                               GARROW_IO_TYPE_MEMORY_MAPPED_FILE,       \
-                               GArrowIOMemoryMappedFilePrivate))
-
-static void
-garrow_io_memory_mapped_file_finalize(GObject *object)
-{
-  GArrowIOMemoryMappedFilePrivate *priv;
-
-  priv = GARROW_IO_MEMORY_MAPPED_FILE_GET_PRIVATE(object);
-
-  priv->memory_mapped_file = nullptr;
-
-  G_OBJECT_CLASS(garrow_io_memory_mapped_file_parent_class)->finalize(object);
-}
-
-static void
-garrow_io_memory_mapped_file_set_property(GObject *object,
-                                          guint prop_id,
-                                          const GValue *value,
-                                          GParamSpec *pspec)
-{
-  GArrowIOMemoryMappedFilePrivate *priv;
-
-  priv = GARROW_IO_MEMORY_MAPPED_FILE_GET_PRIVATE(object);
-
-  switch (prop_id) {
-  case PROP_MEMORY_MAPPED_FILE:
-    priv->memory_mapped_file =
-      *static_cast<std::shared_ptr<arrow::io::MemoryMappedFile> *>(g_value_get_pointer(value));
-    break;
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_io_memory_mapped_file_get_property(GObject *object,
-                                          guint prop_id,
-                                          GValue *value,
-                                          GParamSpec *pspec)
-{
-  switch (prop_id) {
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_io_memory_mapped_file_init(GArrowIOMemoryMappedFile *object)
-{
-}
-
-static void
-garrow_io_memory_mapped_file_class_init(GArrowIOMemoryMappedFileClass *klass)
-{
-  GObjectClass *gobject_class;
-  GParamSpec *spec;
-
-  gobject_class = G_OBJECT_CLASS(klass);
-
-  gobject_class->finalize     = garrow_io_memory_mapped_file_finalize;
-  gobject_class->set_property = garrow_io_memory_mapped_file_set_property;
-  gobject_class->get_property = garrow_io_memory_mapped_file_get_property;
-
-  spec = g_param_spec_pointer("memory-mapped-file",
-                              "io::MemoryMappedFile",
-                              "The raw std::shared<arrow::io::MemoryMappedFile> *",
-                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
-                                                       G_PARAM_CONSTRUCT_ONLY));
-  g_object_class_install_property(gobject_class, PROP_MEMORY_MAPPED_FILE, spec);
-}
-
-/**
- * garrow_io_memory_mapped_file_open:
- * @path: The path of the memory mapped file.
- * @mode: The mode of the memory mapped file.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full): A newly opened
- *   #GArrowIOMemoryMappedFile or %NULL on error.
- */
-GArrowIOMemoryMappedFile *
-garrow_io_memory_mapped_file_open(const gchar *path,
-                                  GArrowIOFileMode mode,
-                                  GError **error)
-{
-  std::shared_ptr<arrow::io::MemoryMappedFile> arrow_memory_mapped_file;
-  auto status =
-    arrow::io::MemoryMappedFile::Open(std::string(path),
-                                      garrow_io_file_mode_to_raw(mode),
-                                      &arrow_memory_mapped_file);
-  if (status.ok()) {
-    return garrow_io_memory_mapped_file_new_raw(&arrow_memory_mapped_file);
-  } else {
-    std::string context("[io][memory-mapped-file][open]: <");
-    context += path;
-    context += ">";
-    garrow_error_set(error, status, context.c_str());
-    return NULL;
-  }
-}
-
-G_END_DECLS
-
-GArrowIOMemoryMappedFile *
-garrow_io_memory_mapped_file_new_raw(std::shared_ptr<arrow::io::MemoryMappedFile> *arrow_memory_mapped_file)
-{
-  auto memory_mapped_file =
-    GARROW_IO_MEMORY_MAPPED_FILE(g_object_new(GARROW_IO_TYPE_MEMORY_MAPPED_FILE,
-                                              "memory-mapped-file", arrow_memory_mapped_file,
-                                              NULL));
-  return memory_mapped_file;
-}
-
-std::shared_ptr<arrow::io::MemoryMappedFile>
-garrow_io_memory_mapped_file_get_raw(GArrowIOMemoryMappedFile *memory_mapped_file)
-{
-  GArrowIOMemoryMappedFilePrivate *priv;
-
-  priv = GARROW_IO_MEMORY_MAPPED_FILE_GET_PRIVATE(memory_mapped_file);
-  return priv->memory_mapped_file;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-memory-mapped-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-memory-mapped-file.h b/c_glib/arrow-glib/io-memory-mapped-file.h
deleted file mode 100644
index 0d2d6c2..0000000
--- a/c_glib/arrow-glib/io-memory-mapped-file.h
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow-glib/io-file-mode.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_MEMORY_MAPPED_FILE       \
-  (garrow_io_memory_mapped_file_get_type())
-#define GARROW_IO_MEMORY_MAPPED_FILE(obj)                               \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                                    \
-                              GARROW_IO_TYPE_MEMORY_MAPPED_FILE,        \
-                              GArrowIOMemoryMappedFile))
-#define GARROW_IO_MEMORY_MAPPED_FILE_CLASS(klass)               \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
-                           GARROW_IO_TYPE_MEMORY_MAPPED_FILE,   \
-                           GArrowIOMemoryMappedFileClass))
-#define GARROW_IO_IS_MEMORY_MAPPED_FILE(obj)                            \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                                    \
-                              GARROW_IO_TYPE_MEMORY_MAPPED_FILE))
-#define GARROW_IO_IS_MEMORY_MAPPED_FILE_CLASS(klass)            \
-  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
-                           GARROW_IO_TYPE_MEMORY_MAPPED_FILE))
-#define GARROW_IO_MEMORY_MAPPED_FILE_GET_CLASS(obj)             \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_IO_TYPE_MEMORY_MAPPED_FILE, \
-                             GArrowIOMemoryMappedFileClass))
-
-typedef struct _GArrowIOMemoryMappedFile         GArrowIOMemoryMappedFile;
-typedef struct _GArrowIOMemoryMappedFileClass    GArrowIOMemoryMappedFileClass;
-
-/**
- * GArrowIOMemoryMappedFile:
- *
- * It wraps `arrow::io::MemoryMappedFile`.
- */
-struct _GArrowIOMemoryMappedFile
-{
-  /*< private >*/
-  GObject parent_instance;
-};
-
-struct _GArrowIOMemoryMappedFileClass
-{
-  GObjectClass parent_class;
-};
-
-GType garrow_io_memory_mapped_file_get_type(void) G_GNUC_CONST;
-
-GArrowIOMemoryMappedFile *garrow_io_memory_mapped_file_open(const gchar *path,
-                                                            GArrowIOFileMode mode,
-                                                            GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-memory-mapped-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-memory-mapped-file.hpp b/c_glib/arrow-glib/io-memory-mapped-file.hpp
deleted file mode 100644
index b48e05f..0000000
--- a/c_glib/arrow-glib/io-memory-mapped-file.hpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/api.h>
-#include <arrow/io/file.h>
-
-#include <arrow-glib/io-memory-mapped-file.h>
-
-GArrowIOMemoryMappedFile *garrow_io_memory_mapped_file_new_raw(std::shared_ptr<arrow::io::MemoryMappedFile> *arrow_memory_mapped_file);
-std::shared_ptr<arrow::io::MemoryMappedFile> garrow_io_memory_mapped_file_get_raw(GArrowIOMemoryMappedFile *memory_mapped_file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-output-stream.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-output-stream.cpp b/c_glib/arrow-glib/io-output-stream.cpp
deleted file mode 100644
index bdf5587..0000000
--- a/c_glib/arrow-glib/io-output-stream.cpp
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-output-stream.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-output-stream
- * @title: GArrowIOOutputStream
- * @short_description: Stream output interface
- *
- * #GArrowIOOutputStream is an interface for stream output. Stream
- * output is file based and writeable
- */
-
-G_DEFINE_INTERFACE(GArrowIOOutputStream,
-                   garrow_io_output_stream,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_output_stream_default_init (GArrowIOOutputStreamInterface *iface)
-{
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::OutputStream>
-garrow_io_output_stream_get_raw(GArrowIOOutputStream *output_stream)
-{
-  auto *iface = GARROW_IO_OUTPUT_STREAM_GET_IFACE(output_stream);
-  return iface->get_raw(output_stream);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-output-stream.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-output-stream.h b/c_glib/arrow-glib/io-output-stream.h
deleted file mode 100644
index 02478ce..0000000
--- a/c_glib/arrow-glib/io-output-stream.h
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_OUTPUT_STREAM            \
-  (garrow_io_output_stream_get_type())
-#define GARROW_IO_OUTPUT_STREAM(obj)                            \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IO_TYPE_OUTPUT_STREAM,     \
-                              GArrowIOOutputStream))
-#define GARROW_IO_IS_OUTPUT_STREAM(obj)                         \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IO_TYPE_OUTPUT_STREAM))
-#define GARROW_IO_OUTPUT_STREAM_GET_IFACE(obj)                          \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                                 \
-                                 GARROW_IO_TYPE_OUTPUT_STREAM,          \
-                                 GArrowIOOutputStreamInterface))
-
-typedef struct _GArrowIOOutputStream          GArrowIOOutputStream;
-typedef struct _GArrowIOOutputStreamInterface GArrowIOOutputStreamInterface;
-
-GType garrow_io_output_stream_get_type(void) G_GNUC_CONST;
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-output-stream.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-output-stream.hpp b/c_glib/arrow-glib/io-output-stream.hpp
deleted file mode 100644
index f144130..0000000
--- a/c_glib/arrow-glib/io-output-stream.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-output-stream.h>
-
-/**
- * GArrowIOOutputStreamInterface:
- *
- * It wraps `arrow::io::OutputStream`.
- */
-struct _GArrowIOOutputStreamInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::OutputStream> (*get_raw)(GArrowIOOutputStream *file);
-};
-
-std::shared_ptr<arrow::io::OutputStream> garrow_io_output_stream_get_raw(GArrowIOOutputStream *output_stream);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-random-access-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-random-access-file.cpp b/c_glib/arrow-glib/io-random-access-file.cpp
deleted file mode 100644
index 552b879..0000000
--- a/c_glib/arrow-glib/io-random-access-file.cpp
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-random-access-file.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-random-access-file
- * @title: GArrowIORandomAccessFile
- * @short_description: File input interface
- *
- * #GArrowIORandomAccessFile is an interface for file input.
- */
-
-G_DEFINE_INTERFACE(GArrowIORandomAccessFile,
-                   garrow_io_random_access_file,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_random_access_file_default_init (GArrowIORandomAccessFileInterface *iface)
-{
-}
-
-/**
- * garrow_io_random_access_file_get_size:
- * @file: A #GArrowIORandomAccessFile.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: The size of the file.
- */
-guint64
-garrow_io_random_access_file_get_size(GArrowIORandomAccessFile *file,
-                                 GError **error)
-{
-  auto *iface = GARROW_IO_RANDOM_ACCESS_FILE_GET_IFACE(file);
-  auto arrow_random_access_file = iface->get_raw(file);
-  int64_t size;
-
-  auto status = arrow_random_access_file->GetSize(&size);
-  if (status.ok()) {
-    return size;
-  } else {
-    garrow_error_set(error, status, "[io][random-access-file][get-size]");
-    return 0;
-  }
-}
-
-/**
- * garrow_io_random_access_file_get_support_zero_copy:
- * @file: A #GArrowIORandomAccessFile.
- *
- * Returns: Whether zero copy read is supported or not.
- */
-gboolean
-garrow_io_random_access_file_get_support_zero_copy(GArrowIORandomAccessFile *file)
-{
-  auto *iface = GARROW_IO_RANDOM_ACCESS_FILE_GET_IFACE(file);
-  auto arrow_random_access_file = iface->get_raw(file);
-
-  return arrow_random_access_file->supports_zero_copy();
-}
-
-/**
- * garrow_io_random_access_file_read_at:
- * @file: A #GArrowIORandomAccessFile.
- * @position: The read start position.
- * @n_bytes: The number of bytes to be read.
- * @n_read_bytes: (out): The read number of bytes.
- * @buffer: (array length=n_bytes): The buffer to be read data.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_io_random_access_file_read_at(GArrowIORandomAccessFile *file,
-                                     gint64 position,
-                                     gint64 n_bytes,
-                                     gint64 *n_read_bytes,
-                                     guint8 *buffer,
-                                     GError **error)
-{
-  const auto arrow_random_access_file =
-    garrow_io_random_access_file_get_raw(file);
-
-  auto status = arrow_random_access_file->ReadAt(position,
-                                                 n_bytes,
-                                                 n_read_bytes,
-                                                 buffer);
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[io][random-access-file][read-at]");
-    return FALSE;
-  }
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::RandomAccessFile>
-garrow_io_random_access_file_get_raw(GArrowIORandomAccessFile *random_access_file)
-{
-  auto *iface = GARROW_IO_RANDOM_ACCESS_FILE_GET_IFACE(random_access_file);
-  return iface->get_raw(random_access_file);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-random-access-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-random-access-file.h b/c_glib/arrow-glib/io-random-access-file.h
deleted file mode 100644
index 8ac63e4..0000000
--- a/c_glib/arrow-glib/io-random-access-file.h
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_RANDOM_ACCESS_FILE       \
-  (garrow_io_random_access_file_get_type())
-#define GARROW_IO_RANDOM_ACCESS_FILE(obj)                            \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                                 \
-                              GARROW_IO_TYPE_RANDOM_ACCESS_FILE,     \
-                              GArrowIORandomAccessFile))
-#define GARROW_IO_IS_RANDOM_ACCESS_FILE(obj)                            \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                                    \
-                              GARROW_IO_TYPE_RANDOM_ACCESS_FILE))
-#define GARROW_IO_RANDOM_ACCESS_FILE_GET_IFACE(obj)                     \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                                 \
-                                 GARROW_IO_TYPE_RANDOM_ACCESS_FILE,     \
-                                 GArrowIORandomAccessFileInterface))
-
-typedef struct _GArrowIORandomAccessFile          GArrowIORandomAccessFile;
-typedef struct _GArrowIORandomAccessFileInterface GArrowIORandomAccessFileInterface;
-
-GType garrow_io_random_access_file_get_type(void) G_GNUC_CONST;
-
-guint64 garrow_io_random_access_file_get_size(GArrowIORandomAccessFile *file,
-                                              GError **error);
-gboolean garrow_io_random_access_file_get_support_zero_copy(GArrowIORandomAccessFile *file);
-gboolean garrow_io_random_access_file_read_at(GArrowIORandomAccessFile *file,
-                                              gint64 position,
-                                              gint64 n_bytes,
-                                              gint64 *n_read_bytes,
-                                              guint8 *buffer,
-                                              GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-random-access-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-random-access-file.hpp b/c_glib/arrow-glib/io-random-access-file.hpp
deleted file mode 100644
index 7c97c9e..0000000
--- a/c_glib/arrow-glib/io-random-access-file.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-random-access-file.h>
-
-/**
- * GArrowIORandomAccessFileInterface:
- *
- * It wraps `arrow::io::RandomAccessFile`.
- */
-struct _GArrowIORandomAccessFileInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::RandomAccessFile> (*get_raw)(GArrowIORandomAccessFile *file);
-};
-
-std::shared_ptr<arrow::io::RandomAccessFile> garrow_io_random_access_file_get_raw(GArrowIORandomAccessFile *random_access_file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-readable.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-readable.cpp b/c_glib/arrow-glib/io-readable.cpp
deleted file mode 100644
index b372a66..0000000
--- a/c_glib/arrow-glib/io-readable.cpp
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-readable.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-readable
- * @title: GArrowIOReadable
- * @short_description: Input interface
- *
- * #GArrowIOReadable is an interface for input. Input must be
- * readable.
- */
-
-G_DEFINE_INTERFACE(GArrowIOReadable,
-                   garrow_io_readable,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_readable_default_init (GArrowIOReadableInterface *iface)
-{
-}
-
-/**
- * garrow_io_readable_read:
- * @readable: A #GArrowIOReadable.
- * @n_bytes: The number of bytes to be read.
- * @n_read_bytes: (out): The read number of bytes.
- * @buffer: (array length=n_bytes): The buffer to be read data.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_io_readable_read(GArrowIOReadable *readable,
-                        gint64 n_bytes,
-                        gint64 *n_read_bytes,
-                        guint8 *buffer,
-                        GError **error)
-{
-  const auto arrow_readable = garrow_io_readable_get_raw(readable);
-
-  auto status = arrow_readable->Read(n_bytes, n_read_bytes, buffer);
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[io][readable][read]");
-    return FALSE;
-  }
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::Readable>
-garrow_io_readable_get_raw(GArrowIOReadable *readable)
-{
-  auto *iface = GARROW_IO_READABLE_GET_IFACE(readable);
-  return iface->get_raw(readable);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-readable.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-readable.h b/c_glib/arrow-glib/io-readable.h
deleted file mode 100644
index 279984b..0000000
--- a/c_glib/arrow-glib/io-readable.h
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_READABLE                 \
-  (garrow_io_readable_get_type())
-#define GARROW_IO_READABLE(obj)                                 \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IO_TYPE_READABLE,          \
-                              GArrowIOReadable))
-#define GARROW_IO_IS_READABLE(obj)                      \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                    \
-                              GARROW_IO_TYPE_READABLE))
-#define GARROW_IO_READABLE_GET_IFACE(obj)                       \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
-                                 GARROW_IO_TYPE_READABLE,       \
-                                 GArrowIOReadableInterface))
-
-typedef struct _GArrowIOReadable          GArrowIOReadable;
-typedef struct _GArrowIOReadableInterface GArrowIOReadableInterface;
-
-GType garrow_io_readable_get_type(void) G_GNUC_CONST;
-
-gboolean garrow_io_readable_read(GArrowIOReadable *readable,
-                                 gint64 n_bytes,
-                                 gint64 *n_read_bytes,
-                                 guint8 *buffer,
-                                 GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-readable.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-readable.hpp b/c_glib/arrow-glib/io-readable.hpp
deleted file mode 100644
index 3d27b3f..0000000
--- a/c_glib/arrow-glib/io-readable.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-readable.h>
-
-/**
- * GArrowIOReadableInterface:
- *
- * It wraps `arrow::io::Readable`.
- */
-struct _GArrowIOReadableInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::Readable> (*get_raw)(GArrowIOReadable *file);
-};
-
-std::shared_ptr<arrow::io::Readable> garrow_io_readable_get_raw(GArrowIOReadable *readable);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-writeable-file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-writeable-file.cpp b/c_glib/arrow-glib/io-writeable-file.cpp
deleted file mode 100644
index 41b682a..0000000
--- a/c_glib/arrow-glib/io-writeable-file.cpp
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-writeable-file.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-writeable-file
- * @title: GArrowIOWriteableFile
- * @short_description: File output interface
- *
- * #GArrowIOWriteableFile is an interface for file output.
- */
-
-G_DEFINE_INTERFACE(GArrowIOWriteableFile,
-                   garrow_io_writeable_file,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_writeable_file_default_init (GArrowIOWriteableFileInterface *iface)
-{
-}
-
-/**
- * garrow_io_writeable_file_write_at:
- * @writeable_file: A #GArrowIOWriteableFile.
- * @position: The write start position.
- * @data: (array length=n_bytes): The data to be written.
- * @n_bytes: The number of bytes to be written.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_io_writeable_file_write_at(GArrowIOWriteableFile *writeable_file,
-                                  gint64 position,
-                                  const guint8 *data,
-                                  gint64 n_bytes,
-                                  GError **error)
-{
-  const auto arrow_writeable_file =
-    garrow_io_writeable_file_get_raw(writeable_file);
-
-  auto status = arrow_writeable_file->WriteAt(position, data, n_bytes);
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[io][writeable-file][write-at]");
-    return FALSE;
-  }
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::WriteableFile>
-garrow_io_writeable_file_get_raw(GArrowIOWriteableFile *writeable_file)
-{
-  auto *iface = GARROW_IO_WRITEABLE_FILE_GET_IFACE(writeable_file);
-  return iface->get_raw(writeable_file);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-writeable-file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-writeable-file.h b/c_glib/arrow-glib/io-writeable-file.h
deleted file mode 100644
index d1ebdbe..0000000
--- a/c_glib/arrow-glib/io-writeable-file.h
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_WRITEABLE_FILE           \
-  (garrow_io_writeable_file_get_type())
-#define GARROW_IO_WRITEABLE_FILE(obj)                           \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IO_TYPE_WRITEABLE_FILE,    \
-                              GArrowIOWriteableFile))
-#define GARROW_IO_IS_WRITEABLE_FILE(obj)                        \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IO_TYPE_WRITEABLE_FILE))
-#define GARROW_IO_WRITEABLE_FILE_GET_IFACE(obj)                         \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                                 \
-                                 GARROW_IO_TYPE_WRITEABLE_FILE,         \
-                                 GArrowIOWriteableFileInterface))
-
-typedef struct _GArrowIOWriteableFile          GArrowIOWriteableFile;
-typedef struct _GArrowIOWriteableFileInterface GArrowIOWriteableFileInterface;
-
-GType garrow_io_writeable_file_get_type(void) G_GNUC_CONST;
-
-gboolean garrow_io_writeable_file_write_at(GArrowIOWriteableFile *writeable_file,
-                                           gint64 position,
-                                           const guint8 *data,
-                                           gint64 n_bytes,
-                                           GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-writeable-file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-writeable-file.hpp b/c_glib/arrow-glib/io-writeable-file.hpp
deleted file mode 100644
index aba95b2..0000000
--- a/c_glib/arrow-glib/io-writeable-file.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-writeable-file.h>
-
-/**
- * GArrowIOWriteableFile:
- *
- * It wraps `arrow::io::WriteableFile`.
- */
-struct _GArrowIOWriteableFileInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::WriteableFile> (*get_raw)(GArrowIOWriteableFile *file);
-};
-
-std::shared_ptr<arrow::io::WriteableFile> garrow_io_writeable_file_get_raw(GArrowIOWriteableFile *writeable_file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-writeable.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-writeable.cpp b/c_glib/arrow-glib/io-writeable.cpp
deleted file mode 100644
index 9ea69e3..0000000
--- a/c_glib/arrow-glib/io-writeable.cpp
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-writeable.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-writeable
- * @title: GArrowIOWriteable
- * @short_description: Output interface
- *
- * #GArrowIOWriteable is an interface for output. Output must be
- * writeable.
- */
-
-G_DEFINE_INTERFACE(GArrowIOWriteable,
-                   garrow_io_writeable,
-                   G_TYPE_OBJECT)
-
-static void
-garrow_io_writeable_default_init (GArrowIOWriteableInterface *iface)
-{
-}
-
-/**
- * garrow_io_writeable_write:
- * @writeable: A #GArrowIOWriteable.
- * @data: (array length=n_bytes): The data to be written.
- * @n_bytes: The number of bytes to be written.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_io_writeable_write(GArrowIOWriteable *writeable,
-                          const guint8 *data,
-                          gint64 n_bytes,
-                          GError **error)
-{
-  const auto arrow_writeable = garrow_io_writeable_get_raw(writeable);
-
-  auto status = arrow_writeable->Write(data, n_bytes);
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[io][writeable][write]");
-    return FALSE;
-  }
-}
-
-/**
- * garrow_io_writeable_flush:
- * @writeable: A #GArrowIOWriteable.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * It ensures writing all data on memory to storage.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_io_writeable_flush(GArrowIOWriteable *writeable,
-                          GError **error)
-{
-  const auto arrow_writeable = garrow_io_writeable_get_raw(writeable);
-
-  auto status = arrow_writeable->Flush();
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[io][writeable][flush]");
-    return FALSE;
-  }
-}
-
-G_END_DECLS
-
-std::shared_ptr<arrow::io::Writeable>
-garrow_io_writeable_get_raw(GArrowIOWriteable *writeable)
-{
-  auto *iface = GARROW_IO_WRITEABLE_GET_IFACE(writeable);
-  return iface->get_raw(writeable);
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-writeable.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-writeable.h b/c_glib/arrow-glib/io-writeable.h
deleted file mode 100644
index ce23247..0000000
--- a/c_glib/arrow-glib/io-writeable.h
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_WRITEABLE                \
-  (garrow_io_writeable_get_type())
-#define GARROW_IO_WRITEABLE(obj)                                \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IO_TYPE_WRITEABLE,         \
-                              GArrowIOWriteable))
-#define GARROW_IO_IS_WRITEABLE(obj)                             \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IO_TYPE_WRITEABLE))
-#define GARROW_IO_WRITEABLE_GET_IFACE(obj)                      \
-  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
-                                 GARROW_IO_TYPE_WRITEABLE,      \
-                                 GArrowIOWriteableInterface))
-
-typedef struct _GArrowIOWriteable          GArrowIOWriteable;
-typedef struct _GArrowIOWriteableInterface GArrowIOWriteableInterface;
-
-GType garrow_io_writeable_get_type(void) G_GNUC_CONST;
-
-gboolean garrow_io_writeable_write(GArrowIOWriteable *writeable,
-                                   const guint8 *data,
-                                   gint64 n_bytes,
-                                   GError **error);
-gboolean garrow_io_writeable_flush(GArrowIOWriteable *writeable,
-                                   GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-writeable.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-writeable.hpp b/c_glib/arrow-glib/io-writeable.hpp
deleted file mode 100644
index f833924..0000000
--- a/c_glib/arrow-glib/io-writeable.hpp
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-writeable.h>
-
-/**
- * GArrowIOWriteableInterface:
- *
- * It wraps `arrow::io::Writeable`.
- */
-struct _GArrowIOWriteableInterface
-{
-  GTypeInterface parent_iface;
-
-  std::shared_ptr<arrow::io::Writeable> (*get_raw)(GArrowIOWriteable *file);
-};
-
-std::shared_ptr<arrow::io::Writeable> garrow_io_writeable_get_raw(GArrowIOWriteable *writeable);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-file-reader.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-file-reader.cpp b/c_glib/arrow-glib/ipc-file-reader.cpp
deleted file mode 100644
index 223be85..0000000
--- a/c_glib/arrow-glib/ipc-file-reader.cpp
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/record-batch.hpp>
-#include <arrow-glib/schema.hpp>
-
-#include <arrow-glib/io-random-access-file.hpp>
-
-#include <arrow-glib/ipc-file-reader.hpp>
-#include <arrow-glib/ipc-metadata-version.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: ipc-file-reader
- * @short_description: File reader class
- *
- * #GArrowIPCFileReader is a class for receiving data by file based IPC.
- */
-
-typedef struct GArrowIPCFileReaderPrivate_ {
-  std::shared_ptr<arrow::ipc::FileReader> file_reader;
-} GArrowIPCFileReaderPrivate;
-
-enum {
-  PROP_0,
-  PROP_FILE_READER
-};
-
-G_DEFINE_TYPE_WITH_PRIVATE(GArrowIPCFileReader,
-                           garrow_ipc_file_reader,
-                           G_TYPE_OBJECT);
-
-#define GARROW_IPC_FILE_READER_GET_PRIVATE(obj)                         \
-  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                                   \
-                               GARROW_IPC_TYPE_FILE_READER,             \
-                               GArrowIPCFileReaderPrivate))
-
-static void
-garrow_ipc_file_reader_finalize(GObject *object)
-{
-  GArrowIPCFileReaderPrivate *priv;
-
-  priv = GARROW_IPC_FILE_READER_GET_PRIVATE(object);
-
-  priv->file_reader = nullptr;
-
-  G_OBJECT_CLASS(garrow_ipc_file_reader_parent_class)->finalize(object);
-}
-
-static void
-garrow_ipc_file_reader_set_property(GObject *object,
-                                    guint prop_id,
-                                    const GValue *value,
-                                    GParamSpec *pspec)
-{
-  GArrowIPCFileReaderPrivate *priv;
-
-  priv = GARROW_IPC_FILE_READER_GET_PRIVATE(object);
-
-  switch (prop_id) {
-  case PROP_FILE_READER:
-    priv->file_reader =
-      *static_cast<std::shared_ptr<arrow::ipc::FileReader> *>(g_value_get_pointer(value));
-    break;
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_ipc_file_reader_get_property(GObject *object,
-                                          guint prop_id,
-                                          GValue *value,
-                                          GParamSpec *pspec)
-{
-  switch (prop_id) {
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_ipc_file_reader_init(GArrowIPCFileReader *object)
-{
-}
-
-static void
-garrow_ipc_file_reader_class_init(GArrowIPCFileReaderClass *klass)
-{
-  GObjectClass *gobject_class;
-  GParamSpec *spec;
-
-  gobject_class = G_OBJECT_CLASS(klass);
-
-  gobject_class->finalize     = garrow_ipc_file_reader_finalize;
-  gobject_class->set_property = garrow_ipc_file_reader_set_property;
-  gobject_class->get_property = garrow_ipc_file_reader_get_property;
-
-  spec = g_param_spec_pointer("file-reader",
-                              "ipc::FileReader",
-                              "The raw std::shared<arrow::ipc::FileReader> *",
-                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
-                                                       G_PARAM_CONSTRUCT_ONLY));
-  g_object_class_install_property(gobject_class, PROP_FILE_READER, spec);
-}
-
-/**
- * garrow_ipc_file_reader_open:
- * @file: The file to be read.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full): A newly opened
- *   #GArrowIPCFileReader or %NULL on error.
- */
-GArrowIPCFileReader *
-garrow_ipc_file_reader_open(GArrowIORandomAccessFile *file,
-                            GError **error)
-{
-  std::shared_ptr<arrow::ipc::FileReader> arrow_file_reader;
-  auto status =
-    arrow::ipc::FileReader::Open(garrow_io_random_access_file_get_raw(file),
-                                 &arrow_file_reader);
-  if (status.ok()) {
-    return garrow_ipc_file_reader_new_raw(&arrow_file_reader);
-  } else {
-    garrow_error_set(error, status, "[ipc][file-reader][open]");
-    return NULL;
-  }
-}
-
-/**
- * garrow_ipc_file_reader_get_schema:
- * @file_reader: A #GArrowIPCFileReader.
- *
- * Returns: (transfer full): The schema in the file.
- */
-GArrowSchema *
-garrow_ipc_file_reader_get_schema(GArrowIPCFileReader *file_reader)
-{
-  auto arrow_file_reader =
-    garrow_ipc_file_reader_get_raw(file_reader);
-  auto arrow_schema = arrow_file_reader->schema();
-  return garrow_schema_new_raw(&arrow_schema);
-}
-
-/**
- * garrow_ipc_file_reader_get_n_record_batches:
- * @file_reader: A #GArrowIPCFileReader.
- *
- * Returns: The number of record batches in the file.
- */
-guint
-garrow_ipc_file_reader_get_n_record_batches(GArrowIPCFileReader *file_reader)
-{
-  auto arrow_file_reader =
-    garrow_ipc_file_reader_get_raw(file_reader);
-  return arrow_file_reader->num_record_batches();
-}
-
-/**
- * garrow_ipc_file_reader_get_version:
- * @file_reader: A #GArrowIPCFileReader.
- *
- * Returns: The format version in the file.
- */
-GArrowIPCMetadataVersion
-garrow_ipc_file_reader_get_version(GArrowIPCFileReader *file_reader)
-{
-  auto arrow_file_reader =
-    garrow_ipc_file_reader_get_raw(file_reader);
-  auto arrow_version = arrow_file_reader->version();
-  return garrow_ipc_metadata_version_from_raw(arrow_version);
-}
-
-/**
- * garrow_ipc_file_reader_get_record_batch:
- * @file_reader: A #GArrowIPCFileReader.
- * @i: The index of the target record batch.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full):
- *   The i-th record batch in the file or %NULL on error.
- */
-GArrowRecordBatch *
-garrow_ipc_file_reader_get_record_batch(GArrowIPCFileReader *file_reader,
-                                        guint i,
-                                        GError **error)
-{
-  auto arrow_file_reader =
-    garrow_ipc_file_reader_get_raw(file_reader);
-  std::shared_ptr<arrow::RecordBatch> arrow_record_batch;
-  auto status = arrow_file_reader->GetRecordBatch(i, &arrow_record_batch);
-
-  if (status.ok()) {
-    return garrow_record_batch_new_raw(&arrow_record_batch);
-  } else {
-    garrow_error_set(error, status, "[ipc][file-reader][get-record-batch]");
-    return NULL;
-  }
-}
-
-G_END_DECLS
-
-GArrowIPCFileReader *
-garrow_ipc_file_reader_new_raw(std::shared_ptr<arrow::ipc::FileReader> *arrow_file_reader)
-{
-  auto file_reader =
-    GARROW_IPC_FILE_READER(g_object_new(GARROW_IPC_TYPE_FILE_READER,
-                                        "file-reader", arrow_file_reader,
-                                        NULL));
-  return file_reader;
-}
-
-std::shared_ptr<arrow::ipc::FileReader>
-garrow_ipc_file_reader_get_raw(GArrowIPCFileReader *file_reader)
-{
-  GArrowIPCFileReaderPrivate *priv;
-
-  priv = GARROW_IPC_FILE_READER_GET_PRIVATE(file_reader);
-  return priv->file_reader;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-file-reader.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-file-reader.h b/c_glib/arrow-glib/ipc-file-reader.h
deleted file mode 100644
index 15eba8e..0000000
--- a/c_glib/arrow-glib/ipc-file-reader.h
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow-glib/record-batch.h>
-#include <arrow-glib/schema.h>
-
-#include <arrow-glib/io-random-access-file.h>
-
-#include <arrow-glib/ipc-metadata-version.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IPC_TYPE_FILE_READER      \
-  (garrow_ipc_file_reader_get_type())
-#define GARROW_IPC_FILE_READER(obj)                             \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IPC_TYPE_FILE_READER,      \
-                              GArrowIPCFileReader))
-#define GARROW_IPC_FILE_READER_CLASS(klass)             \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                     \
-                           GARROW_IPC_TYPE_FILE_READER, \
-                           GArrowIPCFileReaderClass))
-#define GARROW_IPC_IS_FILE_READER(obj)                          \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IPC_TYPE_FILE_READER))
-#define GARROW_IPC_IS_FILE_READER_CLASS(klass)                  \
-  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
-                           GARROW_IPC_TYPE_FILE_READER))
-#define GARROW_IPC_FILE_READER_GET_CLASS(obj)                   \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_IPC_TYPE_FILE_READER,       \
-                             GArrowIPCFileReaderClass))
-
-typedef struct _GArrowIPCFileReader         GArrowIPCFileReader;
-typedef struct _GArrowIPCFileReaderClass    GArrowIPCFileReaderClass;
-
-/**
- * GArrowIPCFileReader:
- *
- * It wraps `arrow::ipc::FileReader`.
- */
-struct _GArrowIPCFileReader
-{
-  /*< private >*/
-  GObject parent_instance;
-};
-
-struct _GArrowIPCFileReaderClass
-{
-  GObjectClass parent_class;
-};
-
-GType garrow_ipc_file_reader_get_type(void) G_GNUC_CONST;
-
-GArrowIPCFileReader *garrow_ipc_file_reader_open(GArrowIORandomAccessFile *file,
-                                                 GError **error);
-
-GArrowSchema *garrow_ipc_file_reader_get_schema(GArrowIPCFileReader *file_reader);
-guint garrow_ipc_file_reader_get_n_record_batches(GArrowIPCFileReader *file_reader);
-GArrowIPCMetadataVersion garrow_ipc_file_reader_get_version(GArrowIPCFileReader *file_reader);
-GArrowRecordBatch *garrow_ipc_file_reader_get_record_batch(GArrowIPCFileReader *file_reader,
-                                                           guint i,
-                                                           GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-file-reader.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-file-reader.hpp b/c_glib/arrow-glib/ipc-file-reader.hpp
deleted file mode 100644
index 66cd45d..0000000
--- a/c_glib/arrow-glib/ipc-file-reader.hpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/api.h>
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/ipc-file-reader.h>
-
-GArrowIPCFileReader *garrow_ipc_file_reader_new_raw(std::shared_ptr<arrow::ipc::FileReader> *arrow_file_reader);
-std::shared_ptr<arrow::ipc::FileReader> garrow_ipc_file_reader_get_raw(GArrowIPCFileReader *file_reader);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-file-writer.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-file-writer.cpp b/c_glib/arrow-glib/ipc-file-writer.cpp
deleted file mode 100644
index d8b3c2e..0000000
--- a/c_glib/arrow-glib/ipc-file-writer.cpp
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/ipc/api.h>
-
-#include <arrow-glib/array.hpp>
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/record-batch.hpp>
-#include <arrow-glib/schema.hpp>
-
-#include <arrow-glib/io-output-stream.hpp>
-
-#include <arrow-glib/ipc-stream-writer.hpp>
-#include <arrow-glib/ipc-file-writer.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: ipc-file-writer
- * @short_description: File writer class
- *
- * #GArrowIPCFileWriter is a class for sending data by file based IPC.
- */
-
-G_DEFINE_TYPE(GArrowIPCFileWriter,
-              garrow_ipc_file_writer,
-              GARROW_IPC_TYPE_STREAM_WRITER);
-
-static void
-garrow_ipc_file_writer_init(GArrowIPCFileWriter *object)
-{
-}
-
-static void
-garrow_ipc_file_writer_class_init(GArrowIPCFileWriterClass *klass)
-{
-}
-
-/**
- * garrow_ipc_file_writer_open:
- * @sink: The output of the writer.
- * @schema: The schema of the writer.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full): A newly opened
- *   #GArrowIPCFileWriter or %NULL on error.
- */
-GArrowIPCFileWriter *
-garrow_ipc_file_writer_open(GArrowIOOutputStream *sink,
-                            GArrowSchema *schema,
-                            GError **error)
-{
-  std::shared_ptr<arrow::ipc::FileWriter> arrow_file_writer;
-  auto status =
-    arrow::ipc::FileWriter::Open(garrow_io_output_stream_get_raw(sink).get(),
-                                 garrow_schema_get_raw(schema),
-                                 &arrow_file_writer);
-  if (status.ok()) {
-    return garrow_ipc_file_writer_new_raw(&arrow_file_writer);
-  } else {
-    garrow_error_set(error, status, "[ipc][file-writer][open]");
-    return NULL;
-  }
-}
-
-/**
- * garrow_ipc_file_writer_write_record_batch:
- * @file_writer: A #GArrowIPCFileWriter.
- * @record_batch: The record batch to be written.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_ipc_file_writer_write_record_batch(GArrowIPCFileWriter *file_writer,
-                                          GArrowRecordBatch *record_batch,
-                                          GError **error)
-{
-  auto arrow_file_writer =
-    garrow_ipc_file_writer_get_raw(file_writer);
-  auto arrow_record_batch =
-    garrow_record_batch_get_raw(record_batch);
-  auto arrow_record_batch_raw =
-    arrow_record_batch.get();
-
-  auto status = arrow_file_writer->WriteRecordBatch(*arrow_record_batch_raw);
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[ipc][file-writer][write-record-batch]");
-    return FALSE;
-  }
-}
-
-/**
- * garrow_ipc_file_writer_close:
- * @file_writer: A #GArrowIPCFileWriter.
- * @error: (nullable): Return locatipcn for a #GError or %NULL.
- *
- * Returns: %TRUE on success, %FALSE if there was an error.
- */
-gboolean
-garrow_ipc_file_writer_close(GArrowIPCFileWriter *file_writer,
-                             GError **error)
-{
-  auto arrow_file_writer =
-    garrow_ipc_file_writer_get_raw(file_writer);
-
-  auto status = arrow_file_writer->Close();
-  if (status.ok()) {
-    return TRUE;
-  } else {
-    garrow_error_set(error, status, "[ipc][file-writer][close]");
-    return FALSE;
-  }
-}
-
-G_END_DECLS
-
-GArrowIPCFileWriter *
-garrow_ipc_file_writer_new_raw(std::shared_ptr<arrow::ipc::FileWriter> *arrow_file_writer)
-{
-  auto file_writer =
-    GARROW_IPC_FILE_WRITER(g_object_new(GARROW_IPC_TYPE_FILE_WRITER,
-                                        "stream-writer", arrow_file_writer,
-                                        NULL));
-  return file_writer;
-}
-
-arrow::ipc::FileWriter *
-garrow_ipc_file_writer_get_raw(GArrowIPCFileWriter *file_writer)
-{
-  auto arrow_stream_writer =
-    garrow_ipc_stream_writer_get_raw(GARROW_IPC_STREAM_WRITER(file_writer));
-  auto arrow_file_writer_raw =
-    dynamic_cast<arrow::ipc::FileWriter *>(arrow_stream_writer.get());
-  return arrow_file_writer_raw;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/ipc-file-writer.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/ipc-file-writer.h b/c_glib/arrow-glib/ipc-file-writer.h
deleted file mode 100644
index 732d942..0000000
--- a/c_glib/arrow-glib/ipc-file-writer.h
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow-glib/ipc-stream-writer.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IPC_TYPE_FILE_WRITER             \
-  (garrow_ipc_file_writer_get_type())
-#define GARROW_IPC_FILE_WRITER(obj)                             \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
-                              GARROW_IPC_TYPE_FILE_WRITER,      \
-                              GArrowIPCFileWriter))
-#define GARROW_IPC_FILE_WRITER_CLASS(klass)             \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                     \
-                           GARROW_IPC_TYPE_FILE_WRITER, \
-                           GArrowIPCFileWriterClass))
-#define GARROW_IPC_IS_FILE_WRITER(obj)                          \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
-                              GARROW_IPC_TYPE_FILE_WRITER))
-#define GARROW_IPC_IS_FILE_WRITER_CLASS(klass)                  \
-  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
-                           GARROW_IPC_TYPE_FILE_WRITER))
-#define GARROW_IPC_FILE_WRITER_GET_CLASS(obj)                   \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_IPC_TYPE_FILE_WRITER,       \
-                             GArrowIPCFileWriterClass))
-
-typedef struct _GArrowIPCFileWriter         GArrowIPCFileWriter;
-typedef struct _GArrowIPCFileWriterClass    GArrowIPCFileWriterClass;
-
-/**
- * GArrowIPCFileWriter:
- *
- * It wraps `arrow::ipc::FileWriter`.
- */
-struct _GArrowIPCFileWriter
-{
-  /*< private >*/
-  GArrowIPCStreamWriter parent_instance;
-};
-
-struct _GArrowIPCFileWriterClass
-{
-  GObjectClass parent_class;
-};
-
-GType garrow_ipc_file_writer_get_type(void) G_GNUC_CONST;
-
-GArrowIPCFileWriter *garrow_ipc_file_writer_open(GArrowIOOutputStream *sink,
-                                                 GArrowSchema *schema,
-                                                 GError **error);
-
-gboolean garrow_ipc_file_writer_write_record_batch(GArrowIPCFileWriter *file_writer,
-                                                   GArrowRecordBatch *record_batch,
-                                                   GError **error);
-gboolean garrow_ipc_file_writer_close(GArrowIPCFileWriter *file_writer,
-                                      GError **error);
-
-G_END_DECLS


[4/4] arrow git commit: ARROW-810: [GLib] Remove io/ipc prefix

Posted by we...@apache.org.
ARROW-810: [GLib] Remove io/ipc prefix

Author: Kouhei Sutou <ko...@clear-code.com>

Closes #530 from kou/glib-remove-io-ipc-prefix and squashes the following commits:

adfad7c [Kouhei Sutou] [GLib] Remove io/ipc prefix


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/6443b828
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/6443b828
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/6443b828

Branch: refs/heads/master
Commit: 6443b82878489ed6a308d1e5ace33088788a060e
Parents: 5e5a587
Author: Kouhei Sutou <ko...@clear-code.com>
Authored: Wed Apr 12 10:54:42 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Wed Apr 12 10:54:42 2017 -0400

----------------------------------------------------------------------
 c_glib/arrow-glib/Makefile.am               |  90 +++----
 c_glib/arrow-glib/arrow-glib.h              |  30 +--
 c_glib/arrow-glib/arrow-glib.hpp            |  28 +--
 c_glib/arrow-glib/file-mode.cpp             |  63 +++++
 c_glib/arrow-glib/file-mode.h               |  40 ++++
 c_glib/arrow-glib/file-mode.hpp             |  27 +++
 c_glib/arrow-glib/file-output-stream.cpp    | 231 ++++++++++++++++++
 c_glib/arrow-glib/file-output-stream.h      |  72 ++++++
 c_glib/arrow-glib/file-output-stream.hpp    |  28 +++
 c_glib/arrow-glib/file-reader.cpp           | 247 +++++++++++++++++++
 c_glib/arrow-glib/file-reader.h             |  83 +++++++
 c_glib/arrow-glib/file-reader.hpp           |  28 +++
 c_glib/arrow-glib/file-writer.cpp           | 158 +++++++++++++
 c_glib/arrow-glib/file-writer.h             |  78 ++++++
 c_glib/arrow-glib/file-writer.hpp           |  28 +++
 c_glib/arrow-glib/file.cpp                  | 116 +++++++++
 c_glib/arrow-glib/file.h                    |  51 ++++
 c_glib/arrow-glib/file.hpp                  |  38 +++
 c_glib/arrow-glib/input-stream.cpp          |  56 +++++
 c_glib/arrow-glib/input-stream.h            |  45 ++++
 c_glib/arrow-glib/input-stream.hpp          |  38 +++
 c_glib/arrow-glib/io-file-mode.cpp          |  63 -----
 c_glib/arrow-glib/io-file-mode.h            |  40 ----
 c_glib/arrow-glib/io-file-mode.hpp          |  27 ---
 c_glib/arrow-glib/io-file-output-stream.cpp | 231 ------------------
 c_glib/arrow-glib/io-file-output-stream.h   |  72 ------
 c_glib/arrow-glib/io-file-output-stream.hpp |  28 ---
 c_glib/arrow-glib/io-file.cpp               | 116 ---------
 c_glib/arrow-glib/io-file.h                 |  51 ----
 c_glib/arrow-glib/io-file.hpp               |  38 ---
 c_glib/arrow-glib/io-input-stream.cpp       |  56 -----
 c_glib/arrow-glib/io-input-stream.h         |  45 ----
 c_glib/arrow-glib/io-input-stream.hpp       |  38 ---
 c_glib/arrow-glib/io-memory-mapped-file.cpp | 287 -----------------------
 c_glib/arrow-glib/io-memory-mapped-file.h   |  72 ------
 c_glib/arrow-glib/io-memory-mapped-file.hpp |  28 ---
 c_glib/arrow-glib/io-output-stream.cpp      |  56 -----
 c_glib/arrow-glib/io-output-stream.h        |  45 ----
 c_glib/arrow-glib/io-output-stream.hpp      |  38 ---
 c_glib/arrow-glib/io-random-access-file.cpp | 128 ----------
 c_glib/arrow-glib/io-random-access-file.h   |  55 -----
 c_glib/arrow-glib/io-random-access-file.hpp |  38 ---
 c_glib/arrow-glib/io-readable.cpp           |  84 -------
 c_glib/arrow-glib/io-readable.h             |  51 ----
 c_glib/arrow-glib/io-readable.hpp           |  38 ---
 c_glib/arrow-glib/io-writeable-file.cpp     |  84 -------
 c_glib/arrow-glib/io-writeable-file.h       |  51 ----
 c_glib/arrow-glib/io-writeable-file.hpp     |  38 ---
 c_glib/arrow-glib/io-writeable.cpp          | 106 ---------
 c_glib/arrow-glib/io-writeable.h            |  52 ----
 c_glib/arrow-glib/io-writeable.hpp          |  38 ---
 c_glib/arrow-glib/ipc-file-reader.cpp       | 247 -------------------
 c_glib/arrow-glib/ipc-file-reader.h         |  83 -------
 c_glib/arrow-glib/ipc-file-reader.hpp       |  28 ---
 c_glib/arrow-glib/ipc-file-writer.cpp       | 158 -------------
 c_glib/arrow-glib/ipc-file-writer.h         |  78 ------
 c_glib/arrow-glib/ipc-file-writer.hpp       |  28 ---
 c_glib/arrow-glib/ipc-metadata-version.cpp  |  63 -----
 c_glib/arrow-glib/ipc-metadata-version.h    |  41 ----
 c_glib/arrow-glib/ipc-metadata-version.hpp  |  27 ---
 c_glib/arrow-glib/ipc-stream-reader.cpp     | 221 -----------------
 c_glib/arrow-glib/ipc-stream-reader.h       |  80 -------
 c_glib/arrow-glib/ipc-stream-reader.hpp     |  28 ---
 c_glib/arrow-glib/ipc-stream-writer.cpp     | 232 ------------------
 c_glib/arrow-glib/ipc-stream-writer.h       |  82 -------
 c_glib/arrow-glib/ipc-stream-writer.hpp     |  28 ---
 c_glib/arrow-glib/memory-mapped-file.cpp    | 287 +++++++++++++++++++++++
 c_glib/arrow-glib/memory-mapped-file.h      |  72 ++++++
 c_glib/arrow-glib/memory-mapped-file.hpp    |  28 +++
 c_glib/arrow-glib/metadata-version.cpp      |  63 +++++
 c_glib/arrow-glib/metadata-version.h        |  41 ++++
 c_glib/arrow-glib/metadata-version.hpp      |  27 +++
 c_glib/arrow-glib/output-stream.cpp         |  56 +++++
 c_glib/arrow-glib/output-stream.h           |  45 ++++
 c_glib/arrow-glib/output-stream.hpp         |  38 +++
 c_glib/arrow-glib/random-access-file.cpp    | 128 ++++++++++
 c_glib/arrow-glib/random-access-file.h      |  55 +++++
 c_glib/arrow-glib/random-access-file.hpp    |  38 +++
 c_glib/arrow-glib/readable.cpp              |  84 +++++++
 c_glib/arrow-glib/readable.h                |  51 ++++
 c_glib/arrow-glib/readable.hpp              |  38 +++
 c_glib/arrow-glib/stream-reader.cpp         | 221 +++++++++++++++++
 c_glib/arrow-glib/stream-reader.h           |  80 +++++++
 c_glib/arrow-glib/stream-reader.hpp         |  28 +++
 c_glib/arrow-glib/stream-writer.cpp         | 232 ++++++++++++++++++
 c_glib/arrow-glib/stream-writer.h           |  82 +++++++
 c_glib/arrow-glib/stream-writer.hpp         |  28 +++
 c_glib/arrow-glib/writeable-file.cpp        |  84 +++++++
 c_glib/arrow-glib/writeable-file.h          |  51 ++++
 c_glib/arrow-glib/writeable-file.hpp        |  38 +++
 c_glib/arrow-glib/writeable.cpp             | 106 +++++++++
 c_glib/arrow-glib/writeable.h               |  52 ++++
 c_glib/arrow-glib/writeable.hpp             |  38 +++
 c_glib/doc/reference/Makefile.am            |   2 +-
 c_glib/doc/reference/arrow-glib-docs.sgml   |  62 ++---
 c_glib/example/read-batch.c                 |  18 +-
 c_glib/example/read-stream.c                |  16 +-
 c_glib/test/test-file-output-stream.rb      |  38 +++
 c_glib/test/test-file-writer.rb             |  45 ++++
 c_glib/test/test-io-file-output-stream.rb   |  38 ---
 c_glib/test/test-io-memory-mapped-file.rb   | 138 -----------
 c_glib/test/test-ipc-file-writer.rb         |  45 ----
 c_glib/test/test-ipc-stream-writer.rb       |  53 -----
 c_glib/test/test-memory-mapped-file.rb      | 138 +++++++++++
 c_glib/test/test-stream-writer.rb           |  53 +++++
 105 files changed, 3915 insertions(+), 3915 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/Makefile.am
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/Makefile.am b/c_glib/arrow-glib/Makefile.am
index e719ccc..387707c 100644
--- a/c_glib/arrow-glib/Makefile.am
+++ b/c_glib/arrow-glib/Makefile.am
@@ -102,23 +102,23 @@ libarrow_glib_la_headers =			\
 	uint64-data-type.h
 
 libarrow_glib_la_headers +=			\
-	io-file.h				\
-	io-file-mode.h				\
-	io-file-output-stream.h			\
-	io-input-stream.h			\
-	io-memory-mapped-file.h			\
-	io-output-stream.h			\
-	io-random-access-file.h			\
-	io-readable.h				\
-	io-writeable.h				\
-	io-writeable-file.h
+	file.h					\
+	file-mode.h				\
+	file-output-stream.h			\
+	input-stream.h				\
+	memory-mapped-file.h			\
+	output-stream.h				\
+	random-access-file.h			\
+	readable.h				\
+	writeable.h				\
+	writeable-file.h
 
 libarrow_glib_la_headers +=			\
-	ipc-file-reader.h			\
-	ipc-file-writer.h			\
-	ipc-stream-reader.h			\
-	ipc-stream-writer.h			\
-	ipc-metadata-version.h
+	file-reader.h				\
+	file-writer.h				\
+	stream-reader.h				\
+	stream-writer.h				\
+	metadata-version.h
 
 libarrow_glib_la_generated_headers =		\
 	enums.h
@@ -190,23 +190,23 @@ libarrow_glib_la_sources =			\
 	$(libarrow_glib_la_generated_sources)
 
 libarrow_glib_la_sources +=			\
-	io-file.cpp				\
-	io-file-mode.cpp			\
-	io-file-output-stream.cpp		\
-	io-input-stream.cpp			\
-	io-memory-mapped-file.cpp		\
-	io-output-stream.cpp			\
-	io-random-access-file.cpp		\
-	io-readable.cpp				\
-	io-writeable.cpp			\
-	io-writeable-file.cpp
+	file.cpp				\
+	file-mode.cpp				\
+	file-output-stream.cpp			\
+	input-stream.cpp			\
+	memory-mapped-file.cpp			\
+	output-stream.cpp			\
+	random-access-file.cpp			\
+	readable.cpp				\
+	writeable.cpp				\
+	writeable-file.cpp
 
 libarrow_glib_la_sources +=			\
-	ipc-file-reader.cpp			\
-	ipc-file-writer.cpp			\
-	ipc-metadata-version.cpp		\
-	ipc-stream-reader.cpp			\
-	ipc-stream-writer.cpp
+	file-reader.cpp				\
+	file-writer.cpp				\
+	metadata-version.cpp			\
+	stream-reader.cpp			\
+	stream-writer.cpp
 
 libarrow_glib_la_cpp_headers =			\
 	array.hpp				\
@@ -223,23 +223,23 @@ libarrow_glib_la_cpp_headers =			\
 	type.hpp
 
 libarrow_glib_la_cpp_headers +=			\
-	io-file.hpp				\
-	io-file-mode.hpp			\
-	io-file-output-stream.hpp		\
-	io-input-stream.hpp			\
-	io-memory-mapped-file.hpp		\
-	io-output-stream.hpp			\
-	io-random-access-file.hpp		\
-	io-readable.hpp				\
-	io-writeable.hpp			\
-	io-writeable-file.hpp
+	file.hpp				\
+	file-mode.hpp				\
+	file-output-stream.hpp			\
+	input-stream.hpp			\
+	memory-mapped-file.hpp			\
+	output-stream.hpp			\
+	random-access-file.hpp			\
+	readable.hpp				\
+	writeable.hpp				\
+	writeable-file.hpp
 
 libarrow_glib_la_cpp_headers +=			\
-	ipc-file-reader.hpp			\
-	ipc-file-writer.hpp			\
-	ipc-metadata-version.hpp		\
-	ipc-stream-reader.hpp			\
-	ipc-stream-writer.hpp
+	file-reader.hpp				\
+	file-writer.hpp				\
+	metadata-version.hpp			\
+	stream-reader.hpp			\
+	stream-writer.hpp
 
 libarrow_glib_la_SOURCES =			\
 	$(libarrow_glib_la_sources)		\

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/arrow-glib.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/arrow-glib.h b/c_glib/arrow-glib/arrow-glib.h
index 9b03175..b15c56f 100644
--- a/c_glib/arrow-glib/arrow-glib.h
+++ b/c_glib/arrow-glib/arrow-glib.h
@@ -79,19 +79,19 @@
 #include <arrow-glib/uint64-array-builder.h>
 #include <arrow-glib/uint64-data-type.h>
 
-#include <arrow-glib/io-file.h>
-#include <arrow-glib/io-file-mode.h>
-#include <arrow-glib/io-file-output-stream.h>
-#include <arrow-glib/io-input-stream.h>
-#include <arrow-glib/io-memory-mapped-file.h>
-#include <arrow-glib/io-output-stream.h>
-#include <arrow-glib/io-random-access-file.h>
-#include <arrow-glib/io-readable.h>
-#include <arrow-glib/io-writeable.h>
-#include <arrow-glib/io-writeable-file.h>
+#include <arrow-glib/file.h>
+#include <arrow-glib/file-mode.h>
+#include <arrow-glib/file-output-stream.h>
+#include <arrow-glib/input-stream.h>
+#include <arrow-glib/memory-mapped-file.h>
+#include <arrow-glib/output-stream.h>
+#include <arrow-glib/random-access-file.h>
+#include <arrow-glib/readable.h>
+#include <arrow-glib/writeable.h>
+#include <arrow-glib/writeable-file.h>
 
-#include <arrow-glib/ipc-file-reader.h>
-#include <arrow-glib/ipc-file-writer.h>
-#include <arrow-glib/ipc-metadata-version.h>
-#include <arrow-glib/ipc-stream-reader.h>
-#include <arrow-glib/ipc-stream-writer.h>
+#include <arrow-glib/file-reader.h>
+#include <arrow-glib/file-writer.h>
+#include <arrow-glib/metadata-version.h>
+#include <arrow-glib/stream-reader.h>
+#include <arrow-glib/stream-writer.h>

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/arrow-glib.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/arrow-glib.hpp b/c_glib/arrow-glib/arrow-glib.hpp
index fd59d4a..3404d4d 100644
--- a/c_glib/arrow-glib/arrow-glib.hpp
+++ b/c_glib/arrow-glib/arrow-glib.hpp
@@ -36,18 +36,18 @@
 #include <arrow-glib/table.hpp>
 #include <arrow-glib/type.hpp>
 
-#include <arrow-glib/io-file.hpp>
-#include <arrow-glib/io-file-mode.hpp>
-#include <arrow-glib/io-file-output-stream.hpp>
-#include <arrow-glib/io-input-stream.hpp>
-#include <arrow-glib/io-memory-mapped-file.hpp>
-#include <arrow-glib/io-output-stream.hpp>
-#include <arrow-glib/io-random-access-file.hpp>
-#include <arrow-glib/io-readable.hpp>
-#include <arrow-glib/io-writeable.hpp>
+#include <arrow-glib/file.hpp>
+#include <arrow-glib/file-mode.hpp>
+#include <arrow-glib/file-output-stream.hpp>
+#include <arrow-glib/input-stream.hpp>
+#include <arrow-glib/memory-mapped-file.hpp>
+#include <arrow-glib/output-stream.hpp>
+#include <arrow-glib/random-access-file.hpp>
+#include <arrow-glib/readable.hpp>
+#include <arrow-glib/writeable.hpp>
 
-#include <arrow-glib/ipc-file-reader.hpp>
-#include <arrow-glib/ipc-file-writer.hpp>
-#include <arrow-glib/ipc-metadata-version.hpp>
-#include <arrow-glib/ipc-stream-reader.hpp>
-#include <arrow-glib/ipc-stream-writer.hpp>
+#include <arrow-glib/file-reader.hpp>
+#include <arrow-glib/file-writer.hpp>
+#include <arrow-glib/metadata-version.hpp>
+#include <arrow-glib/stream-reader.hpp>
+#include <arrow-glib/stream-writer.hpp>

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-mode.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-mode.cpp b/c_glib/arrow-glib/file-mode.cpp
new file mode 100644
index 0000000..1fb1706
--- /dev/null
+++ b/c_glib/arrow-glib/file-mode.cpp
@@ -0,0 +1,63 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow-glib/file-mode.hpp>
+
+/**
+ * SECTION: file-mode
+ * @title: GArrowFileMode
+ * @short_description: File mode mapping between Arrow and arrow-glib
+ *
+ * #GArrowFileMode provides file modes corresponding to
+ * `arrow::io::FileMode::type` values.
+ */
+
+GArrowFileMode
+garrow_file_mode_from_raw(arrow::io::FileMode::type mode)
+{
+  switch (mode) {
+  case arrow::io::FileMode::type::READ:
+    return GARROW_FILE_MODE_READ;
+  case arrow::io::FileMode::type::WRITE:
+    return GARROW_FILE_MODE_WRITE;
+  case arrow::io::FileMode::type::READWRITE:
+    return GARROW_FILE_MODE_READWRITE;
+  default:
+    return GARROW_FILE_MODE_READ;
+  }
+}
+
+arrow::io::FileMode::type
+garrow_file_mode_to_raw(GArrowFileMode mode)
+{
+  switch (mode) {
+  case GARROW_FILE_MODE_READ:
+    return arrow::io::FileMode::type::READ;
+  case GARROW_FILE_MODE_WRITE:
+    return arrow::io::FileMode::type::WRITE;
+  case GARROW_FILE_MODE_READWRITE:
+    return arrow::io::FileMode::type::READWRITE;
+  default:
+    return arrow::io::FileMode::type::READ;
+  }
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-mode.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-mode.h b/c_glib/arrow-glib/file-mode.h
new file mode 100644
index 0000000..8812af8
--- /dev/null
+++ b/c_glib/arrow-glib/file-mode.h
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+/**
+ * GArrowFileMode:
+ * @GARROW_FILE_MODE_READ: For read.
+ * @GARROW_FILE_MODE_WRITE: For write.
+ * @GARROW_FILE_MODE_READWRITE: For read-write.
+ *
+ * They are corresponding to `arrow::io::FileMode::type` values.
+ */
+typedef enum {
+  GARROW_FILE_MODE_READ,
+  GARROW_FILE_MODE_WRITE,
+  GARROW_FILE_MODE_READWRITE
+} GArrowFileMode;
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-mode.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-mode.hpp b/c_glib/arrow-glib/file-mode.hpp
new file mode 100644
index 0000000..2b67379
--- /dev/null
+++ b/c_glib/arrow-glib/file-mode.hpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/file-mode.h>
+
+GArrowFileMode garrow_file_mode_from_raw(arrow::io::FileMode::type mode);
+arrow::io::FileMode::type garrow_file_mode_to_raw(GArrowFileMode mode);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-output-stream.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-output-stream.cpp b/c_glib/arrow-glib/file-output-stream.cpp
new file mode 100644
index 0000000..b6ca42a
--- /dev/null
+++ b/c_glib/arrow-glib/file-output-stream.cpp
@@ -0,0 +1,231 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/io/file.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/file.hpp>
+#include <arrow-glib/file-output-stream.hpp>
+#include <arrow-glib/output-stream.hpp>
+#include <arrow-glib/writeable.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: file-output-stream
+ * @short_description: A file output stream.
+ *
+ * The #GArrowFileOutputStream is a class for file output stream.
+ */
+
+typedef struct GArrowFileOutputStreamPrivate_ {
+  std::shared_ptr<arrow::io::FileOutputStream> file_output_stream;
+} GArrowFileOutputStreamPrivate;
+
+enum {
+  PROP_0,
+  PROP_FILE_OUTPUT_STREAM
+};
+
+static std::shared_ptr<arrow::io::FileInterface>
+garrow_file_output_stream_get_raw_file_interface(GArrowFile *file)
+{
+  auto file_output_stream = GARROW_FILE_OUTPUT_STREAM(file);
+  auto arrow_file_output_stream =
+    garrow_file_output_stream_get_raw(file_output_stream);
+  return arrow_file_output_stream;
+}
+
+static void
+garrow_file_interface_init(GArrowFileInterface *iface)
+{
+  iface->get_raw = garrow_file_output_stream_get_raw_file_interface;
+}
+
+static std::shared_ptr<arrow::io::Writeable>
+garrow_file_output_stream_get_raw_writeable_interface(GArrowWriteable *writeable)
+{
+  auto file_output_stream = GARROW_FILE_OUTPUT_STREAM(writeable);
+  auto arrow_file_output_stream =
+    garrow_file_output_stream_get_raw(file_output_stream);
+  return arrow_file_output_stream;
+}
+
+static void
+garrow_writeable_interface_init(GArrowWriteableInterface *iface)
+{
+  iface->get_raw = garrow_file_output_stream_get_raw_writeable_interface;
+}
+
+static std::shared_ptr<arrow::io::OutputStream>
+garrow_file_output_stream_get_raw_output_stream_interface(GArrowOutputStream *output_stream)
+{
+  auto file_output_stream = GARROW_FILE_OUTPUT_STREAM(output_stream);
+  auto arrow_file_output_stream =
+    garrow_file_output_stream_get_raw(file_output_stream);
+  return arrow_file_output_stream;
+}
+
+static void
+garrow_output_stream_interface_init(GArrowOutputStreamInterface *iface)
+{
+  iface->get_raw = garrow_file_output_stream_get_raw_output_stream_interface;
+}
+
+G_DEFINE_TYPE_WITH_CODE(GArrowFileOutputStream,
+                        garrow_file_output_stream,
+                        G_TYPE_OBJECT,
+                        G_ADD_PRIVATE(GArrowFileOutputStream)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_FILE,
+                                              garrow_file_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_WRITEABLE,
+                                              garrow_writeable_interface_init)
+                        G_IMPLEMENT_INTERFACE(GARROW_TYPE_OUTPUT_STREAM,
+                                              garrow_output_stream_interface_init));
+
+#define GARROW_FILE_OUTPUT_STREAM_GET_PRIVATE(obj)                   \
+  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                                   \
+                               GARROW_TYPE_FILE_OUTPUT_STREAM,       \
+                               GArrowFileOutputStreamPrivate))
+
+static void
+garrow_file_output_stream_finalize(GObject *object)
+{
+  GArrowFileOutputStreamPrivate *priv;
+
+  priv = GARROW_FILE_OUTPUT_STREAM_GET_PRIVATE(object);
+
+  priv->file_output_stream = nullptr;
+
+  G_OBJECT_CLASS(garrow_file_output_stream_parent_class)->finalize(object);
+}
+
+static void
+garrow_file_output_stream_set_property(GObject *object,
+                                          guint prop_id,
+                                          const GValue *value,
+                                          GParamSpec *pspec)
+{
+  GArrowFileOutputStreamPrivate *priv;
+
+  priv = GARROW_FILE_OUTPUT_STREAM_GET_PRIVATE(object);
+
+  switch (prop_id) {
+  case PROP_FILE_OUTPUT_STREAM:
+    priv->file_output_stream =
+      *static_cast<std::shared_ptr<arrow::io::FileOutputStream> *>(g_value_get_pointer(value));
+    break;
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_file_output_stream_get_property(GObject *object,
+                                          guint prop_id,
+                                          GValue *value,
+                                          GParamSpec *pspec)
+{
+  switch (prop_id) {
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_file_output_stream_init(GArrowFileOutputStream *object)
+{
+}
+
+static void
+garrow_file_output_stream_class_init(GArrowFileOutputStreamClass *klass)
+{
+  GObjectClass *gobject_class;
+  GParamSpec *spec;
+
+  gobject_class = G_OBJECT_CLASS(klass);
+
+  gobject_class->finalize     = garrow_file_output_stream_finalize;
+  gobject_class->set_property = garrow_file_output_stream_set_property;
+  gobject_class->get_property = garrow_file_output_stream_get_property;
+
+  spec = g_param_spec_pointer("file-output-stream",
+                              "io::FileOutputStream",
+                              "The raw std::shared<arrow::io::FileOutputStream> *",
+                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
+                                                       G_PARAM_CONSTRUCT_ONLY));
+  g_object_class_install_property(gobject_class, PROP_FILE_OUTPUT_STREAM, spec);
+}
+
+/**
+ * garrow_file_output_stream_open:
+ * @path: The path of the file output stream.
+ * @append: Whether the path is opened as append mode or recreate mode.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full): A newly opened
+ *   #GArrowFileOutputStream or %NULL on error.
+ */
+GArrowFileOutputStream *
+garrow_file_output_stream_open(const gchar *path,
+                                  gboolean append,
+                                  GError **error)
+{
+  std::shared_ptr<arrow::io::FileOutputStream> arrow_file_output_stream;
+  auto status =
+    arrow::io::FileOutputStream::Open(std::string(path),
+                                      append,
+                                      &arrow_file_output_stream);
+  if (status.ok()) {
+    return garrow_file_output_stream_new_raw(&arrow_file_output_stream);
+  } else {
+    std::string context("[io][file-output-stream][open]: <");
+    context += path;
+    context += ">";
+    garrow_error_set(error, status, context.c_str());
+    return NULL;
+  }
+}
+
+G_END_DECLS
+
+GArrowFileOutputStream *
+garrow_file_output_stream_new_raw(std::shared_ptr<arrow::io::FileOutputStream> *arrow_file_output_stream)
+{
+  auto file_output_stream =
+    GARROW_FILE_OUTPUT_STREAM(g_object_new(GARROW_TYPE_FILE_OUTPUT_STREAM,
+                                              "file-output-stream", arrow_file_output_stream,
+                                              NULL));
+  return file_output_stream;
+}
+
+std::shared_ptr<arrow::io::FileOutputStream>
+garrow_file_output_stream_get_raw(GArrowFileOutputStream *file_output_stream)
+{
+  GArrowFileOutputStreamPrivate *priv;
+
+  priv = GARROW_FILE_OUTPUT_STREAM_GET_PRIVATE(file_output_stream);
+  return priv->file_output_stream;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-output-stream.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-output-stream.h b/c_glib/arrow-glib/file-output-stream.h
new file mode 100644
index 0000000..bef3700
--- /dev/null
+++ b/c_glib/arrow-glib/file-output-stream.h
@@ -0,0 +1,72 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_FILE_OUTPUT_STREAM       \
+  (garrow_file_output_stream_get_type())
+#define GARROW_FILE_OUTPUT_STREAM(obj)                               \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                                    \
+                              GARROW_TYPE_FILE_OUTPUT_STREAM,        \
+                              GArrowFileOutputStream))
+#define GARROW_FILE_OUTPUT_STREAM_CLASS(klass)               \
+  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
+                           GARROW_TYPE_FILE_OUTPUT_STREAM,   \
+                           GArrowFileOutputStreamClass))
+#define GARROW_IS_FILE_OUTPUT_STREAM(obj)                            \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                                    \
+                              GARROW_TYPE_FILE_OUTPUT_STREAM))
+#define GARROW_IS_FILE_OUTPUT_STREAM_CLASS(klass)            \
+  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
+                           GARROW_TYPE_FILE_OUTPUT_STREAM))
+#define GARROW_FILE_OUTPUT_STREAM_GET_CLASS(obj)             \
+  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
+                             GARROW_TYPE_FILE_OUTPUT_STREAM, \
+                             GArrowFileOutputStreamClass))
+
+typedef struct _GArrowFileOutputStream         GArrowFileOutputStream;
+typedef struct _GArrowFileOutputStreamClass    GArrowFileOutputStreamClass;
+
+/**
+ * GArrowFileOutputStream:
+ *
+ * It wraps `arrow::io::FileOutputStream`.
+ */
+struct _GArrowFileOutputStream
+{
+  /*< private >*/
+  GObject parent_instance;
+};
+
+struct _GArrowFileOutputStreamClass
+{
+  GObjectClass parent_class;
+};
+
+GType garrow_file_output_stream_get_type(void) G_GNUC_CONST;
+
+GArrowFileOutputStream *garrow_file_output_stream_open(const gchar *path,
+                                                            gboolean append,
+                                                            GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-output-stream.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-output-stream.hpp b/c_glib/arrow-glib/file-output-stream.hpp
new file mode 100644
index 0000000..0b10418
--- /dev/null
+++ b/c_glib/arrow-glib/file-output-stream.hpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/api.h>
+#include <arrow/io/file.h>
+
+#include <arrow-glib/file-output-stream.h>
+
+GArrowFileOutputStream *garrow_file_output_stream_new_raw(std::shared_ptr<arrow::io::FileOutputStream> *arrow_file_output_stream);
+std::shared_ptr<arrow::io::FileOutputStream> garrow_file_output_stream_get_raw(GArrowFileOutputStream *file_output_stream);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-reader.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-reader.cpp b/c_glib/arrow-glib/file-reader.cpp
new file mode 100644
index 0000000..c2aeabe
--- /dev/null
+++ b/c_glib/arrow-glib/file-reader.cpp
@@ -0,0 +1,247 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/record-batch.hpp>
+#include <arrow-glib/schema.hpp>
+
+#include <arrow-glib/random-access-file.hpp>
+
+#include <arrow-glib/file-reader.hpp>
+#include <arrow-glib/metadata-version.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: file-reader
+ * @short_description: File reader class
+ *
+ * #GArrowFileReader is a class for receiving data by file based IPC.
+ */
+
+typedef struct GArrowFileReaderPrivate_ {
+  std::shared_ptr<arrow::ipc::FileReader> file_reader;
+} GArrowFileReaderPrivate;
+
+enum {
+  PROP_0,
+  PROP_FILE_READER
+};
+
+G_DEFINE_TYPE_WITH_PRIVATE(GArrowFileReader,
+                           garrow_file_reader,
+                           G_TYPE_OBJECT);
+
+#define GARROW_FILE_READER_GET_PRIVATE(obj)                         \
+  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                                   \
+                               GARROW_TYPE_FILE_READER,             \
+                               GArrowFileReaderPrivate))
+
+static void
+garrow_file_reader_finalize(GObject *object)
+{
+  GArrowFileReaderPrivate *priv;
+
+  priv = GARROW_FILE_READER_GET_PRIVATE(object);
+
+  priv->file_reader = nullptr;
+
+  G_OBJECT_CLASS(garrow_file_reader_parent_class)->finalize(object);
+}
+
+static void
+garrow_file_reader_set_property(GObject *object,
+                                    guint prop_id,
+                                    const GValue *value,
+                                    GParamSpec *pspec)
+{
+  GArrowFileReaderPrivate *priv;
+
+  priv = GARROW_FILE_READER_GET_PRIVATE(object);
+
+  switch (prop_id) {
+  case PROP_FILE_READER:
+    priv->file_reader =
+      *static_cast<std::shared_ptr<arrow::ipc::FileReader> *>(g_value_get_pointer(value));
+    break;
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_file_reader_get_property(GObject *object,
+                                          guint prop_id,
+                                          GValue *value,
+                                          GParamSpec *pspec)
+{
+  switch (prop_id) {
+  default:
+    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
+    break;
+  }
+}
+
+static void
+garrow_file_reader_init(GArrowFileReader *object)
+{
+}
+
+static void
+garrow_file_reader_class_init(GArrowFileReaderClass *klass)
+{
+  GObjectClass *gobject_class;
+  GParamSpec *spec;
+
+  gobject_class = G_OBJECT_CLASS(klass);
+
+  gobject_class->finalize     = garrow_file_reader_finalize;
+  gobject_class->set_property = garrow_file_reader_set_property;
+  gobject_class->get_property = garrow_file_reader_get_property;
+
+  spec = g_param_spec_pointer("file-reader",
+                              "ipc::FileReader",
+                              "The raw std::shared<arrow::ipc::FileReader> *",
+                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
+                                                       G_PARAM_CONSTRUCT_ONLY));
+  g_object_class_install_property(gobject_class, PROP_FILE_READER, spec);
+}
+
+/**
+ * garrow_file_reader_open:
+ * @file: The file to be read.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full): A newly opened
+ *   #GArrowFileReader or %NULL on error.
+ */
+GArrowFileReader *
+garrow_file_reader_open(GArrowRandomAccessFile *file,
+                            GError **error)
+{
+  std::shared_ptr<arrow::ipc::FileReader> arrow_file_reader;
+  auto status =
+    arrow::ipc::FileReader::Open(garrow_random_access_file_get_raw(file),
+                                 &arrow_file_reader);
+  if (status.ok()) {
+    return garrow_file_reader_new_raw(&arrow_file_reader);
+  } else {
+    garrow_error_set(error, status, "[ipc][file-reader][open]");
+    return NULL;
+  }
+}
+
+/**
+ * garrow_file_reader_get_schema:
+ * @file_reader: A #GArrowFileReader.
+ *
+ * Returns: (transfer full): The schema in the file.
+ */
+GArrowSchema *
+garrow_file_reader_get_schema(GArrowFileReader *file_reader)
+{
+  auto arrow_file_reader =
+    garrow_file_reader_get_raw(file_reader);
+  auto arrow_schema = arrow_file_reader->schema();
+  return garrow_schema_new_raw(&arrow_schema);
+}
+
+/**
+ * garrow_file_reader_get_n_record_batches:
+ * @file_reader: A #GArrowFileReader.
+ *
+ * Returns: The number of record batches in the file.
+ */
+guint
+garrow_file_reader_get_n_record_batches(GArrowFileReader *file_reader)
+{
+  auto arrow_file_reader =
+    garrow_file_reader_get_raw(file_reader);
+  return arrow_file_reader->num_record_batches();
+}
+
+/**
+ * garrow_file_reader_get_version:
+ * @file_reader: A #GArrowFileReader.
+ *
+ * Returns: The format version in the file.
+ */
+GArrowMetadataVersion
+garrow_file_reader_get_version(GArrowFileReader *file_reader)
+{
+  auto arrow_file_reader =
+    garrow_file_reader_get_raw(file_reader);
+  auto arrow_version = arrow_file_reader->version();
+  return garrow_metadata_version_from_raw(arrow_version);
+}
+
+/**
+ * garrow_file_reader_get_record_batch:
+ * @file_reader: A #GArrowFileReader.
+ * @i: The index of the target record batch.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full):
+ *   The i-th record batch in the file or %NULL on error.
+ */
+GArrowRecordBatch *
+garrow_file_reader_get_record_batch(GArrowFileReader *file_reader,
+                                        guint i,
+                                        GError **error)
+{
+  auto arrow_file_reader =
+    garrow_file_reader_get_raw(file_reader);
+  std::shared_ptr<arrow::RecordBatch> arrow_record_batch;
+  auto status = arrow_file_reader->GetRecordBatch(i, &arrow_record_batch);
+
+  if (status.ok()) {
+    return garrow_record_batch_new_raw(&arrow_record_batch);
+  } else {
+    garrow_error_set(error, status, "[ipc][file-reader][get-record-batch]");
+    return NULL;
+  }
+}
+
+G_END_DECLS
+
+GArrowFileReader *
+garrow_file_reader_new_raw(std::shared_ptr<arrow::ipc::FileReader> *arrow_file_reader)
+{
+  auto file_reader =
+    GARROW_FILE_READER(g_object_new(GARROW_TYPE_FILE_READER,
+                                        "file-reader", arrow_file_reader,
+                                        NULL));
+  return file_reader;
+}
+
+std::shared_ptr<arrow::ipc::FileReader>
+garrow_file_reader_get_raw(GArrowFileReader *file_reader)
+{
+  GArrowFileReaderPrivate *priv;
+
+  priv = GARROW_FILE_READER_GET_PRIVATE(file_reader);
+  return priv->file_reader;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-reader.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-reader.h b/c_glib/arrow-glib/file-reader.h
new file mode 100644
index 0000000..084f714
--- /dev/null
+++ b/c_glib/arrow-glib/file-reader.h
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow-glib/record-batch.h>
+#include <arrow-glib/schema.h>
+
+#include <arrow-glib/random-access-file.h>
+
+#include <arrow-glib/metadata-version.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_FILE_READER      \
+  (garrow_file_reader_get_type())
+#define GARROW_FILE_READER(obj)                             \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_FILE_READER,      \
+                              GArrowFileReader))
+#define GARROW_FILE_READER_CLASS(klass)             \
+  (G_TYPE_CHECK_CLASS_CAST((klass),                     \
+                           GARROW_TYPE_FILE_READER, \
+                           GArrowFileReaderClass))
+#define GARROW_IS_FILE_READER(obj)                          \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_FILE_READER))
+#define GARROW_IS_FILE_READER_CLASS(klass)                  \
+  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
+                           GARROW_TYPE_FILE_READER))
+#define GARROW_FILE_READER_GET_CLASS(obj)                   \
+  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
+                             GARROW_TYPE_FILE_READER,       \
+                             GArrowFileReaderClass))
+
+typedef struct _GArrowFileReader         GArrowFileReader;
+typedef struct _GArrowFileReaderClass    GArrowFileReaderClass;
+
+/**
+ * GArrowFileReader:
+ *
+ * It wraps `arrow::ipc::FileReader`.
+ */
+struct _GArrowFileReader
+{
+  /*< private >*/
+  GObject parent_instance;
+};
+
+struct _GArrowFileReaderClass
+{
+  GObjectClass parent_class;
+};
+
+GType garrow_file_reader_get_type(void) G_GNUC_CONST;
+
+GArrowFileReader *garrow_file_reader_open(GArrowRandomAccessFile *file,
+                                                 GError **error);
+
+GArrowSchema *garrow_file_reader_get_schema(GArrowFileReader *file_reader);
+guint garrow_file_reader_get_n_record_batches(GArrowFileReader *file_reader);
+GArrowMetadataVersion garrow_file_reader_get_version(GArrowFileReader *file_reader);
+GArrowRecordBatch *garrow_file_reader_get_record_batch(GArrowFileReader *file_reader,
+                                                           guint i,
+                                                           GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-reader.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-reader.hpp b/c_glib/arrow-glib/file-reader.hpp
new file mode 100644
index 0000000..152379b
--- /dev/null
+++ b/c_glib/arrow-glib/file-reader.hpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/api.h>
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/file-reader.h>
+
+GArrowFileReader *garrow_file_reader_new_raw(std::shared_ptr<arrow::ipc::FileReader> *arrow_file_reader);
+std::shared_ptr<arrow::ipc::FileReader> garrow_file_reader_get_raw(GArrowFileReader *file_reader);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-writer.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-writer.cpp b/c_glib/arrow-glib/file-writer.cpp
new file mode 100644
index 0000000..68eca2e
--- /dev/null
+++ b/c_glib/arrow-glib/file-writer.cpp
@@ -0,0 +1,158 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/array.hpp>
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/record-batch.hpp>
+#include <arrow-glib/schema.hpp>
+
+#include <arrow-glib/output-stream.hpp>
+
+#include <arrow-glib/stream-writer.hpp>
+#include <arrow-glib/file-writer.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: file-writer
+ * @short_description: File writer class
+ *
+ * #GArrowFileWriter is a class for sending data by file based IPC.
+ */
+
+G_DEFINE_TYPE(GArrowFileWriter,
+              garrow_file_writer,
+              GARROW_TYPE_STREAM_WRITER);
+
+static void
+garrow_file_writer_init(GArrowFileWriter *object)
+{
+}
+
+static void
+garrow_file_writer_class_init(GArrowFileWriterClass *klass)
+{
+}
+
+/**
+ * garrow_file_writer_open:
+ * @sink: The output of the writer.
+ * @schema: The schema of the writer.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: (nullable) (transfer full): A newly opened
+ *   #GArrowFileWriter or %NULL on error.
+ */
+GArrowFileWriter *
+garrow_file_writer_open(GArrowOutputStream *sink,
+                            GArrowSchema *schema,
+                            GError **error)
+{
+  std::shared_ptr<arrow::ipc::FileWriter> arrow_file_writer;
+  auto status =
+    arrow::ipc::FileWriter::Open(garrow_output_stream_get_raw(sink).get(),
+                                 garrow_schema_get_raw(schema),
+                                 &arrow_file_writer);
+  if (status.ok()) {
+    return garrow_file_writer_new_raw(&arrow_file_writer);
+  } else {
+    garrow_error_set(error, status, "[ipc][file-writer][open]");
+    return NULL;
+  }
+}
+
+/**
+ * garrow_file_writer_write_record_batch:
+ * @file_writer: A #GArrowFileWriter.
+ * @record_batch: The record batch to be written.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_file_writer_write_record_batch(GArrowFileWriter *file_writer,
+                                          GArrowRecordBatch *record_batch,
+                                          GError **error)
+{
+  auto arrow_file_writer =
+    garrow_file_writer_get_raw(file_writer);
+  auto arrow_record_batch =
+    garrow_record_batch_get_raw(record_batch);
+  auto arrow_record_batch_raw =
+    arrow_record_batch.get();
+
+  auto status = arrow_file_writer->WriteRecordBatch(*arrow_record_batch_raw);
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[ipc][file-writer][write-record-batch]");
+    return FALSE;
+  }
+}
+
+/**
+ * garrow_file_writer_close:
+ * @file_writer: A #GArrowFileWriter.
+ * @error: (nullable): Return locatipcn for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_file_writer_close(GArrowFileWriter *file_writer,
+                             GError **error)
+{
+  auto arrow_file_writer =
+    garrow_file_writer_get_raw(file_writer);
+
+  auto status = arrow_file_writer->Close();
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[ipc][file-writer][close]");
+    return FALSE;
+  }
+}
+
+G_END_DECLS
+
+GArrowFileWriter *
+garrow_file_writer_new_raw(std::shared_ptr<arrow::ipc::FileWriter> *arrow_file_writer)
+{
+  auto file_writer =
+    GARROW_FILE_WRITER(g_object_new(GARROW_TYPE_FILE_WRITER,
+                                        "stream-writer", arrow_file_writer,
+                                        NULL));
+  return file_writer;
+}
+
+arrow::ipc::FileWriter *
+garrow_file_writer_get_raw(GArrowFileWriter *file_writer)
+{
+  auto arrow_stream_writer =
+    garrow_stream_writer_get_raw(GARROW_STREAM_WRITER(file_writer));
+  auto arrow_file_writer_raw =
+    dynamic_cast<arrow::ipc::FileWriter *>(arrow_stream_writer.get());
+  return arrow_file_writer_raw;
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-writer.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-writer.h b/c_glib/arrow-glib/file-writer.h
new file mode 100644
index 0000000..7f9a4f0
--- /dev/null
+++ b/c_glib/arrow-glib/file-writer.h
@@ -0,0 +1,78 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow-glib/stream-writer.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_FILE_WRITER             \
+  (garrow_file_writer_get_type())
+#define GARROW_FILE_WRITER(obj)                             \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_FILE_WRITER,      \
+                              GArrowFileWriter))
+#define GARROW_FILE_WRITER_CLASS(klass)             \
+  (G_TYPE_CHECK_CLASS_CAST((klass),                     \
+                           GARROW_TYPE_FILE_WRITER, \
+                           GArrowFileWriterClass))
+#define GARROW_IS_FILE_WRITER(obj)                          \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_FILE_WRITER))
+#define GARROW_IS_FILE_WRITER_CLASS(klass)                  \
+  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
+                           GARROW_TYPE_FILE_WRITER))
+#define GARROW_FILE_WRITER_GET_CLASS(obj)                   \
+  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
+                             GARROW_TYPE_FILE_WRITER,       \
+                             GArrowFileWriterClass))
+
+typedef struct _GArrowFileWriter         GArrowFileWriter;
+typedef struct _GArrowFileWriterClass    GArrowFileWriterClass;
+
+/**
+ * GArrowFileWriter:
+ *
+ * It wraps `arrow::ipc::FileWriter`.
+ */
+struct _GArrowFileWriter
+{
+  /*< private >*/
+  GArrowStreamWriter parent_instance;
+};
+
+struct _GArrowFileWriterClass
+{
+  GObjectClass parent_class;
+};
+
+GType garrow_file_writer_get_type(void) G_GNUC_CONST;
+
+GArrowFileWriter *garrow_file_writer_open(GArrowOutputStream *sink,
+                                                 GArrowSchema *schema,
+                                                 GError **error);
+
+gboolean garrow_file_writer_write_record_batch(GArrowFileWriter *file_writer,
+                                                   GArrowRecordBatch *record_batch,
+                                                   GError **error);
+gboolean garrow_file_writer_close(GArrowFileWriter *file_writer,
+                                      GError **error);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file-writer.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file-writer.hpp b/c_glib/arrow-glib/file-writer.hpp
new file mode 100644
index 0000000..f6a720a
--- /dev/null
+++ b/c_glib/arrow-glib/file-writer.hpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/api.h>
+#include <arrow/ipc/api.h>
+
+#include <arrow-glib/file-writer.h>
+
+GArrowFileWriter *garrow_file_writer_new_raw(std::shared_ptr<arrow::ipc::FileWriter> *arrow_file_writer);
+arrow::ipc::FileWriter *garrow_file_writer_get_raw(GArrowFileWriter *file_writer);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file.cpp b/c_glib/arrow-glib/file.cpp
new file mode 100644
index 0000000..0d0fe1d
--- /dev/null
+++ b/c_glib/arrow-glib/file.cpp
@@ -0,0 +1,116 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/file.hpp>
+#include <arrow-glib/file-mode.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: file
+ * @title: GArrowFile
+ * @short_description: File interface
+ *
+ * #GArrowFile is an interface for file.
+ */
+
+G_DEFINE_INTERFACE(GArrowFile,
+                   garrow_file,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_file_default_init (GArrowFileInterface *iface)
+{
+}
+
+/**
+ * garrow_file_close:
+ * @file: A #GArrowFile.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: %TRUE on success, %FALSE if there was an error.
+ */
+gboolean
+garrow_file_close(GArrowFile *file,
+                     GError **error)
+{
+  auto arrow_file = garrow_file_get_raw(file);
+
+  auto status = arrow_file->Close();
+  if (status.ok()) {
+    return TRUE;
+  } else {
+    garrow_error_set(error, status, "[io][file][close]");
+    return FALSE;
+  }
+}
+
+/**
+ * garrow_file_tell:
+ * @file: A #GArrowFile.
+ * @error: (nullable): Return location for a #GError or %NULL.
+ *
+ * Returns: The current offset on success, -1 if there was an error.
+ */
+gint64
+garrow_file_tell(GArrowFile *file,
+                    GError **error)
+{
+  auto arrow_file = garrow_file_get_raw(file);
+
+  gint64 position;
+  auto status = arrow_file->Tell(&position);
+  if (status.ok()) {
+    return position;
+  } else {
+    garrow_error_set(error, status, "[io][file][tell]");
+    return -1;
+  }
+}
+
+/**
+ * garrow_file_get_mode:
+ * @file: A #GArrowFile.
+ *
+ * Returns: The mode of the file.
+ */
+GArrowFileMode
+garrow_file_get_mode(GArrowFile *file)
+{
+  auto arrow_file = garrow_file_get_raw(file);
+
+  auto arrow_mode = arrow_file->mode();
+  return garrow_file_mode_from_raw(arrow_mode);
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::FileInterface>
+garrow_file_get_raw(GArrowFile *file)
+{
+  auto *iface = GARROW_FILE_GET_IFACE(file);
+  return iface->get_raw(file);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file.h b/c_glib/arrow-glib/file.h
new file mode 100644
index 0000000..68054aa
--- /dev/null
+++ b/c_glib/arrow-glib/file.h
@@ -0,0 +1,51 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow-glib/file-mode.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_FILE                     \
+  (garrow_file_get_type())
+#define GARROW_FILE(obj)                             \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                    \
+                              GARROW_TYPE_FILE,      \
+                              GArrowFile))
+#define GARROW_IS_FILE(obj)                          \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                    \
+                              GARROW_TYPE_FILE))
+#define GARROW_FILE_GET_IFACE(obj)                           \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
+                                 GARROW_TYPE_FILE,           \
+                                 GArrowFileInterface))
+
+typedef struct _GArrowFile          GArrowFile;
+typedef struct _GArrowFileInterface GArrowFileInterface;
+
+GType garrow_file_get_type(void) G_GNUC_CONST;
+
+gboolean garrow_file_close(GArrowFile *file,
+                              GError **error);
+gint64 garrow_file_tell(GArrowFile *file,
+                           GError **error);
+GArrowFileMode garrow_file_get_mode(GArrowFile *file);
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/file.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/file.hpp b/c_glib/arrow-glib/file.hpp
new file mode 100644
index 0000000..c4cc787
--- /dev/null
+++ b/c_glib/arrow-glib/file.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/file.h>
+
+/**
+ * GArrowFileInterface:
+ *
+ * It wraps `arrow::io::FileInterface`.
+ */
+struct _GArrowFileInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::FileInterface> (*get_raw)(GArrowFile *file);
+};
+
+std::shared_ptr<arrow::io::FileInterface> garrow_file_get_raw(GArrowFile *file);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/input-stream.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/input-stream.cpp b/c_glib/arrow-glib/input-stream.cpp
new file mode 100644
index 0000000..36bef80
--- /dev/null
+++ b/c_glib/arrow-glib/input-stream.cpp
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+#ifdef HAVE_CONFIG_H
+#  include <config.h>
+#endif
+
+#include <arrow/api.h>
+
+#include <arrow-glib/error.hpp>
+#include <arrow-glib/input-stream.hpp>
+
+G_BEGIN_DECLS
+
+/**
+ * SECTION: input-stream
+ * @title: GArrowInputStream
+ * @short_description: Stream input interface
+ *
+ * #GArrowInputStream is an interface for stream input. Stream input
+ * is file based and readable.
+ */
+
+G_DEFINE_INTERFACE(GArrowInputStream,
+                   garrow_input_stream,
+                   G_TYPE_OBJECT)
+
+static void
+garrow_input_stream_default_init (GArrowInputStreamInterface *iface)
+{
+}
+
+G_END_DECLS
+
+std::shared_ptr<arrow::io::InputStream>
+garrow_input_stream_get_raw(GArrowInputStream *input_stream)
+{
+  auto *iface = GARROW_INPUT_STREAM_GET_IFACE(input_stream);
+  return iface->get_raw(input_stream);
+}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/input-stream.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/input-stream.h b/c_glib/arrow-glib/input-stream.h
new file mode 100644
index 0000000..4b331b9
--- /dev/null
+++ b/c_glib/arrow-glib/input-stream.h
@@ -0,0 +1,45 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <glib-object.h>
+
+G_BEGIN_DECLS
+
+#define GARROW_TYPE_INPUT_STREAM             \
+  (garrow_input_stream_get_type())
+#define GARROW_INPUT_STREAM(obj)                             \
+  (G_TYPE_CHECK_INSTANCE_CAST((obj),                            \
+                              GARROW_TYPE_INPUT_STREAM,      \
+                              GArrowInputStream))
+#define GARROW_IS_INPUT_STREAM(obj)                          \
+  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                            \
+                              GARROW_TYPE_INPUT_STREAM))
+#define GARROW_INPUT_STREAM_GET_IFACE(obj)                   \
+  (G_TYPE_INSTANCE_GET_INTERFACE((obj),                         \
+                                 GARROW_TYPE_INPUT_STREAM,   \
+                                 GArrowInputStreamInterface))
+
+typedef struct _GArrowInputStream          GArrowInputStream;
+typedef struct _GArrowInputStreamInterface GArrowInputStreamInterface;
+
+GType garrow_input_stream_get_type(void) G_GNUC_CONST;
+
+G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/input-stream.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/input-stream.hpp b/c_glib/arrow-glib/input-stream.hpp
new file mode 100644
index 0000000..7958df1
--- /dev/null
+++ b/c_glib/arrow-glib/input-stream.hpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+#pragma once
+
+#include <arrow/io/interfaces.h>
+
+#include <arrow-glib/input-stream.h>
+
+/**
+ * GArrowInputStreamInterface:
+ *
+ * It wraps `arrow::io::InputStream`.
+ */
+struct _GArrowInputStreamInterface
+{
+  GTypeInterface parent_iface;
+
+  std::shared_ptr<arrow::io::InputStream> (*get_raw)(GArrowInputStream *file);
+};
+
+std::shared_ptr<arrow::io::InputStream> garrow_input_stream_get_raw(GArrowInputStream *input_stream);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file-mode.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file-mode.cpp b/c_glib/arrow-glib/io-file-mode.cpp
deleted file mode 100644
index 7998d3f..0000000
--- a/c_glib/arrow-glib/io-file-mode.cpp
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow-glib/io-file-mode.hpp>
-
-/**
- * SECTION: io-file-mode
- * @title: GArrowIOFileMode
- * @short_description: File mode mapping between Arrow and arrow-glib
- *
- * #GArrowIOFileMode provides file modes corresponding to
- * `arrow::io::FileMode::type` values.
- */
-
-GArrowIOFileMode
-garrow_io_file_mode_from_raw(arrow::io::FileMode::type mode)
-{
-  switch (mode) {
-  case arrow::io::FileMode::type::READ:
-    return GARROW_IO_FILE_MODE_READ;
-  case arrow::io::FileMode::type::WRITE:
-    return GARROW_IO_FILE_MODE_WRITE;
-  case arrow::io::FileMode::type::READWRITE:
-    return GARROW_IO_FILE_MODE_READWRITE;
-  default:
-    return GARROW_IO_FILE_MODE_READ;
-  }
-}
-
-arrow::io::FileMode::type
-garrow_io_file_mode_to_raw(GArrowIOFileMode mode)
-{
-  switch (mode) {
-  case GARROW_IO_FILE_MODE_READ:
-    return arrow::io::FileMode::type::READ;
-  case GARROW_IO_FILE_MODE_WRITE:
-    return arrow::io::FileMode::type::WRITE;
-  case GARROW_IO_FILE_MODE_READWRITE:
-    return arrow::io::FileMode::type::READWRITE;
-  default:
-    return arrow::io::FileMode::type::READ;
-  }
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file-mode.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file-mode.h b/c_glib/arrow-glib/io-file-mode.h
deleted file mode 100644
index 03eca35..0000000
--- a/c_glib/arrow-glib/io-file-mode.h
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-/**
- * GArrowIOFileMode:
- * @GARROW_IO_FILE_MODE_READ: For read.
- * @GARROW_IO_FILE_MODE_WRITE: For write.
- * @GARROW_IO_FILE_MODE_READWRITE: For read-write.
- *
- * They are corresponding to `arrow::io::FileMode::type` values.
- */
-typedef enum {
-  GARROW_IO_FILE_MODE_READ,
-  GARROW_IO_FILE_MODE_WRITE,
-  GARROW_IO_FILE_MODE_READWRITE
-} GArrowIOFileMode;
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file-mode.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file-mode.hpp b/c_glib/arrow-glib/io-file-mode.hpp
deleted file mode 100644
index b3d8ac6..0000000
--- a/c_glib/arrow-glib/io-file-mode.hpp
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/io/interfaces.h>
-
-#include <arrow-glib/io-file-mode.h>
-
-GArrowIOFileMode garrow_io_file_mode_from_raw(arrow::io::FileMode::type mode);
-arrow::io::FileMode::type garrow_io_file_mode_to_raw(GArrowIOFileMode mode);

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file-output-stream.cpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file-output-stream.cpp b/c_glib/arrow-glib/io-file-output-stream.cpp
deleted file mode 100644
index 673e8cd..0000000
--- a/c_glib/arrow-glib/io-file-output-stream.cpp
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * 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.
- */
-
-#ifdef HAVE_CONFIG_H
-#  include <config.h>
-#endif
-
-#include <arrow/io/file.h>
-
-#include <arrow-glib/error.hpp>
-#include <arrow-glib/io-file.hpp>
-#include <arrow-glib/io-file-output-stream.hpp>
-#include <arrow-glib/io-output-stream.hpp>
-#include <arrow-glib/io-writeable.hpp>
-
-G_BEGIN_DECLS
-
-/**
- * SECTION: io-file-output-stream
- * @short_description: A file output stream.
- *
- * The #GArrowIOFileOutputStream is a class for file output stream.
- */
-
-typedef struct GArrowIOFileOutputStreamPrivate_ {
-  std::shared_ptr<arrow::io::FileOutputStream> file_output_stream;
-} GArrowIOFileOutputStreamPrivate;
-
-enum {
-  PROP_0,
-  PROP_FILE_OUTPUT_STREAM
-};
-
-static std::shared_ptr<arrow::io::FileInterface>
-garrow_io_file_output_stream_get_raw_file_interface(GArrowIOFile *file)
-{
-  auto file_output_stream = GARROW_IO_FILE_OUTPUT_STREAM(file);
-  auto arrow_file_output_stream =
-    garrow_io_file_output_stream_get_raw(file_output_stream);
-  return arrow_file_output_stream;
-}
-
-static void
-garrow_io_file_interface_init(GArrowIOFileInterface *iface)
-{
-  iface->get_raw = garrow_io_file_output_stream_get_raw_file_interface;
-}
-
-static std::shared_ptr<arrow::io::Writeable>
-garrow_io_file_output_stream_get_raw_writeable_interface(GArrowIOWriteable *writeable)
-{
-  auto file_output_stream = GARROW_IO_FILE_OUTPUT_STREAM(writeable);
-  auto arrow_file_output_stream =
-    garrow_io_file_output_stream_get_raw(file_output_stream);
-  return arrow_file_output_stream;
-}
-
-static void
-garrow_io_writeable_interface_init(GArrowIOWriteableInterface *iface)
-{
-  iface->get_raw = garrow_io_file_output_stream_get_raw_writeable_interface;
-}
-
-static std::shared_ptr<arrow::io::OutputStream>
-garrow_io_file_output_stream_get_raw_output_stream_interface(GArrowIOOutputStream *output_stream)
-{
-  auto file_output_stream = GARROW_IO_FILE_OUTPUT_STREAM(output_stream);
-  auto arrow_file_output_stream =
-    garrow_io_file_output_stream_get_raw(file_output_stream);
-  return arrow_file_output_stream;
-}
-
-static void
-garrow_io_output_stream_interface_init(GArrowIOOutputStreamInterface *iface)
-{
-  iface->get_raw = garrow_io_file_output_stream_get_raw_output_stream_interface;
-}
-
-G_DEFINE_TYPE_WITH_CODE(GArrowIOFileOutputStream,
-                        garrow_io_file_output_stream,
-                        G_TYPE_OBJECT,
-                        G_ADD_PRIVATE(GArrowIOFileOutputStream)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_FILE,
-                                              garrow_io_file_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_WRITEABLE,
-                                              garrow_io_writeable_interface_init)
-                        G_IMPLEMENT_INTERFACE(GARROW_IO_TYPE_OUTPUT_STREAM,
-                                              garrow_io_output_stream_interface_init));
-
-#define GARROW_IO_FILE_OUTPUT_STREAM_GET_PRIVATE(obj)                   \
-  (G_TYPE_INSTANCE_GET_PRIVATE((obj),                                   \
-                               GARROW_IO_TYPE_FILE_OUTPUT_STREAM,       \
-                               GArrowIOFileOutputStreamPrivate))
-
-static void
-garrow_io_file_output_stream_finalize(GObject *object)
-{
-  GArrowIOFileOutputStreamPrivate *priv;
-
-  priv = GARROW_IO_FILE_OUTPUT_STREAM_GET_PRIVATE(object);
-
-  priv->file_output_stream = nullptr;
-
-  G_OBJECT_CLASS(garrow_io_file_output_stream_parent_class)->finalize(object);
-}
-
-static void
-garrow_io_file_output_stream_set_property(GObject *object,
-                                          guint prop_id,
-                                          const GValue *value,
-                                          GParamSpec *pspec)
-{
-  GArrowIOFileOutputStreamPrivate *priv;
-
-  priv = GARROW_IO_FILE_OUTPUT_STREAM_GET_PRIVATE(object);
-
-  switch (prop_id) {
-  case PROP_FILE_OUTPUT_STREAM:
-    priv->file_output_stream =
-      *static_cast<std::shared_ptr<arrow::io::FileOutputStream> *>(g_value_get_pointer(value));
-    break;
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_io_file_output_stream_get_property(GObject *object,
-                                          guint prop_id,
-                                          GValue *value,
-                                          GParamSpec *pspec)
-{
-  switch (prop_id) {
-  default:
-    G_OBJECT_WARN_INVALID_PROPERTY_ID(object, prop_id, pspec);
-    break;
-  }
-}
-
-static void
-garrow_io_file_output_stream_init(GArrowIOFileOutputStream *object)
-{
-}
-
-static void
-garrow_io_file_output_stream_class_init(GArrowIOFileOutputStreamClass *klass)
-{
-  GObjectClass *gobject_class;
-  GParamSpec *spec;
-
-  gobject_class = G_OBJECT_CLASS(klass);
-
-  gobject_class->finalize     = garrow_io_file_output_stream_finalize;
-  gobject_class->set_property = garrow_io_file_output_stream_set_property;
-  gobject_class->get_property = garrow_io_file_output_stream_get_property;
-
-  spec = g_param_spec_pointer("file-output-stream",
-                              "io::FileOutputStream",
-                              "The raw std::shared<arrow::io::FileOutputStream> *",
-                              static_cast<GParamFlags>(G_PARAM_WRITABLE |
-                                                       G_PARAM_CONSTRUCT_ONLY));
-  g_object_class_install_property(gobject_class, PROP_FILE_OUTPUT_STREAM, spec);
-}
-
-/**
- * garrow_io_file_output_stream_open:
- * @path: The path of the file output stream.
- * @append: Whether the path is opened as append mode or recreate mode.
- * @error: (nullable): Return location for a #GError or %NULL.
- *
- * Returns: (nullable) (transfer full): A newly opened
- *   #GArrowIOFileOutputStream or %NULL on error.
- */
-GArrowIOFileOutputStream *
-garrow_io_file_output_stream_open(const gchar *path,
-                                  gboolean append,
-                                  GError **error)
-{
-  std::shared_ptr<arrow::io::FileOutputStream> arrow_file_output_stream;
-  auto status =
-    arrow::io::FileOutputStream::Open(std::string(path),
-                                      append,
-                                      &arrow_file_output_stream);
-  if (status.ok()) {
-    return garrow_io_file_output_stream_new_raw(&arrow_file_output_stream);
-  } else {
-    std::string context("[io][file-output-stream][open]: <");
-    context += path;
-    context += ">";
-    garrow_error_set(error, status, context.c_str());
-    return NULL;
-  }
-}
-
-G_END_DECLS
-
-GArrowIOFileOutputStream *
-garrow_io_file_output_stream_new_raw(std::shared_ptr<arrow::io::FileOutputStream> *arrow_file_output_stream)
-{
-  auto file_output_stream =
-    GARROW_IO_FILE_OUTPUT_STREAM(g_object_new(GARROW_IO_TYPE_FILE_OUTPUT_STREAM,
-                                              "file-output-stream", arrow_file_output_stream,
-                                              NULL));
-  return file_output_stream;
-}
-
-std::shared_ptr<arrow::io::FileOutputStream>
-garrow_io_file_output_stream_get_raw(GArrowIOFileOutputStream *file_output_stream)
-{
-  GArrowIOFileOutputStreamPrivate *priv;
-
-  priv = GARROW_IO_FILE_OUTPUT_STREAM_GET_PRIVATE(file_output_stream);
-  return priv->file_output_stream;
-}

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file-output-stream.h
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file-output-stream.h b/c_glib/arrow-glib/io-file-output-stream.h
deleted file mode 100644
index 032b125..0000000
--- a/c_glib/arrow-glib/io-file-output-stream.h
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <glib-object.h>
-
-G_BEGIN_DECLS
-
-#define GARROW_IO_TYPE_FILE_OUTPUT_STREAM       \
-  (garrow_io_file_output_stream_get_type())
-#define GARROW_IO_FILE_OUTPUT_STREAM(obj)                               \
-  (G_TYPE_CHECK_INSTANCE_CAST((obj),                                    \
-                              GARROW_IO_TYPE_FILE_OUTPUT_STREAM,        \
-                              GArrowIOFileOutputStream))
-#define GARROW_IO_FILE_OUTPUT_STREAM_CLASS(klass)               \
-  (G_TYPE_CHECK_CLASS_CAST((klass),                             \
-                           GARROW_IO_TYPE_FILE_OUTPUT_STREAM,   \
-                           GArrowIOFileOutputStreamClass))
-#define GARROW_IO_IS_FILE_OUTPUT_STREAM(obj)                            \
-  (G_TYPE_CHECK_INSTANCE_TYPE((obj),                                    \
-                              GARROW_IO_TYPE_FILE_OUTPUT_STREAM))
-#define GARROW_IO_IS_FILE_OUTPUT_STREAM_CLASS(klass)            \
-  (G_TYPE_CHECK_CLASS_TYPE((klass),                             \
-                           GARROW_IO_TYPE_FILE_OUTPUT_STREAM))
-#define GARROW_IO_FILE_OUTPUT_STREAM_GET_CLASS(obj)             \
-  (G_TYPE_INSTANCE_GET_CLASS((obj),                             \
-                             GARROW_IO_TYPE_FILE_OUTPUT_STREAM, \
-                             GArrowIOFileOutputStreamClass))
-
-typedef struct _GArrowIOFileOutputStream         GArrowIOFileOutputStream;
-typedef struct _GArrowIOFileOutputStreamClass    GArrowIOFileOutputStreamClass;
-
-/**
- * GArrowIOFileOutputStream:
- *
- * It wraps `arrow::io::FileOutputStream`.
- */
-struct _GArrowIOFileOutputStream
-{
-  /*< private >*/
-  GObject parent_instance;
-};
-
-struct _GArrowIOFileOutputStreamClass
-{
-  GObjectClass parent_class;
-};
-
-GType garrow_io_file_output_stream_get_type(void) G_GNUC_CONST;
-
-GArrowIOFileOutputStream *garrow_io_file_output_stream_open(const gchar *path,
-                                                            gboolean append,
-                                                            GError **error);
-
-G_END_DECLS

http://git-wip-us.apache.org/repos/asf/arrow/blob/6443b828/c_glib/arrow-glib/io-file-output-stream.hpp
----------------------------------------------------------------------
diff --git a/c_glib/arrow-glib/io-file-output-stream.hpp b/c_glib/arrow-glib/io-file-output-stream.hpp
deleted file mode 100644
index 76b8e91..0000000
--- a/c_glib/arrow-glib/io-file-output-stream.hpp
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.
- */
-
-#pragma once
-
-#include <arrow/api.h>
-#include <arrow/io/file.h>
-
-#include <arrow-glib/io-file-output-stream.h>
-
-GArrowIOFileOutputStream *garrow_io_file_output_stream_new_raw(std::shared_ptr<arrow::io::FileOutputStream> *arrow_file_output_stream);
-std::shared_ptr<arrow::io::FileOutputStream> garrow_io_file_output_stream_get_raw(GArrowIOFileOutputStream *file_output_stream);