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

zeroshade pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-go.git


The following commit(s) were added to refs/heads/main by this push:
     new 17694053 chore: replace golang.org/x/xerrors for stdlib errors (#763)
17694053 is described below

commit 176940535f1335a6dba98f8830bde49232ef32fb
Author: Sebastiaan van Stijn <[email protected]>
AuthorDate: Mon Apr 13 23:23:14 2026 +0200

    chore: replace golang.org/x/xerrors for stdlib errors (#763)
    
    relates to:
    
    - https://github.com/apache/arrow/pull/5997
    - https://github.com/apache/arrow/issues/16992
    
    
    
    The golang.org/x/xerrors module is a transitional module for what became
    go1.13 errors. Most of its functionality is now available in stdlib
    errors, with the exception of stdlib not providing a stack-trace (and
    some utility functions related to that).
    
    Looking at history to see if that functionality was essential for using
    this package, I found that the golang.org/x/errors module was introduced
    in [apache/arrow@7126fdb], to address [ARROW-7357].
    
    > we should migrate away from `pkg/errors` to `golang.org/x/xerrors` to
    > ensure better error handling (and one that is Go-1.13 compatible).
    
    Based on the above, it looks like the intent was to provide
    compatibility with go1.13+ errors, which are now mainline, so stdlib
    errors should be preferred for this.
    
    [ARROW-7357]: https://issues.apache.org/jira/browse/ARROW-7357
    [apache/arrow@7126fdb]:
    
https://github.com/apache/arrow/commit/7126fdb5d6d2f7dd94fdd3d43ab9256825ff3d44
    
    ### Rationale for this change
    
    
    ### What changes are included in this PR?
    
    
    ### Are these changes tested?
    
    
    ### Are there any user-facing changes?
    
    Signed-off-by: Sebastiaan van Stijn <[email protected]>
    Co-authored-by: Matt Topol <[email protected]>
---
 arrow/cdata/cdata.go                               | 15 ++++----
 arrow/cdata/interface.go                           |  4 +--
 arrow/compute/utils.go                             |  4 +--
 arrow/datatype_fixedwidth.go                       | 20 +++++------
 arrow/internal/flight_integration/scenario.go      |  5 ++-
 arrow/scalar/nested.go                             |  3 +-
 arrow/scalar/scalar.go                             |  6 ++--
 go.mod                                             |  1 -
 go.sum                                             |  2 --
 internal/types/extension_types.go                  |  8 ++---
 parquet/file/column_reader.go                      | 11 +++---
 parquet/file/file_reader.go                        | 18 +++++-----
 parquet/file/file_reader_mmap.go                   |  6 ++--
 parquet/file/level_conversion.go                   |  8 ++---
 parquet/file/page_reader.go                        | 11 +++---
 parquet/file/page_writer.go                        |  6 ++--
 parquet/file/record_reader.go                      | 12 +++----
 parquet/file/row_group_reader.go                   |  6 ++--
 parquet/file/row_group_writer.go                   |  9 ++---
 parquet/internal/encoding/byte_array_decoder.go    |  9 +++--
 parquet/internal/encoding/byte_stream_split.go     |  6 ++--
 parquet/internal/encoding/decoder.go               |  4 +--
 parquet/internal/encoding/delta_byte_array.go      |  5 +--
 .../internal/encoding/delta_length_byte_array.go   |  5 +--
 .../encoding/fixed_len_byte_array_decoder.go       |  9 +++--
 parquet/internal/encoding/types.go                 |  4 +--
 parquet/internal/utils/rle.go                      |  4 +--
 parquet/internal/utils/typed_rle_dict.go           |  6 ++--
 parquet/metadata/column_chunk.go                   |  4 +--
 parquet/metadata/file.go                           |  4 +--
 parquet/pqarrow/file_reader.go                     |  5 ++-
 parquet/pqarrow/file_writer.go                     |  4 +--
 parquet/pqarrow/path_builder.go                    |  4 +--
 parquet/pqarrow/schema.go                          | 42 +++++++++++-----------
 parquet/schema/helpers.go                          | 14 ++++----
 parquet/schema/node.go                             | 12 +++----
 parquet/schema/schema.go                           | 10 +++---
 37 files changed, 150 insertions(+), 156 deletions(-)

diff --git a/arrow/cdata/cdata.go b/arrow/cdata/cdata.go
index d8eeb5b2..352dfd9a 100644
--- a/arrow/cdata/cdata.go
+++ b/arrow/cdata/cdata.go
@@ -54,7 +54,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/array"
        "github.com/apache/arrow-go/v18/arrow/bitutil"
        "github.com/apache/arrow-go/v18/arrow/memory"
-       "golang.org/x/xerrors"
 )
 
 type (
@@ -224,22 +223,22 @@ func importSchema(schema *CArrowSchema) (ret arrow.Field, 
err error) {
                var precision, scale int
 
                if len(propList) < 2 || len(propList) > 3 {
-                       return ret, xerrors.Errorf("invalid decimal spec '%s': 
wrong number of properties", f)
+                       return ret, fmt.Errorf("invalid decimal spec '%s': 
wrong number of properties", f)
                } else if len(propList) == 3 {
                        bitwidth, err = strconv.Atoi(propList[2])
                        if err != nil {
-                               return ret, xerrors.Errorf("could not parse 
decimal bitwidth in '%s': %s", f, err.Error())
+                               return ret, fmt.Errorf("could not parse decimal 
bitwidth in '%s': %w", f, err)
                        }
                }
 
                precision, err = strconv.Atoi(propList[0])
                if err != nil {
-                       return ret, xerrors.Errorf("could not parse decimal 
precision in '%s': %s", f, err.Error())
+                       return ret, fmt.Errorf("could not parse decimal 
precision in '%s': %w", f, err)
                }
 
                scale, err = strconv.Atoi(propList[1])
                if err != nil {
-                       return ret, xerrors.Errorf("could not parse decimal 
scale in '%s': %s", f, err.Error())
+                       return ret, fmt.Errorf("could not parse decimal scale 
in '%s': %w", f, err)
                }
 
                switch bitwidth {
@@ -252,7 +251,7 @@ func importSchema(schema *CArrowSchema) (ret arrow.Field, 
err error) {
                case 256:
                        dt = &arrow.Decimal256Type{Precision: int32(precision), 
Scale: int32(scale)}
                default:
-                       return ret, xerrors.Errorf("unsupported decimal 
bitwidth, got '%s'", f)
+                       return ret, fmt.Errorf("unsupported decimal bitwidth, 
got '%s'", f)
                }
        }
 
@@ -328,7 +327,7 @@ func importSchema(schema *CArrowSchema) (ret arrow.Field, 
err error) {
 
        if dt == nil {
                // if we didn't find a type, then it's something we haven't 
implemented.
-               err = xerrors.New("unimplemented type")
+               err = errors.New("unimplemented type")
        } else {
                ret.Type = dt
        }
@@ -797,7 +796,7 @@ func (imp *cimporter) importFixedSizePrimitive() error {
                values, err = imp.importFixedSizeBuffer(1, 
bitutil.BytesForBits(int64(fw.BitWidth())))
        } else {
                if fw.BitWidth() != 1 {
-                       return xerrors.New("invalid bitwidth")
+                       return errors.New("invalid bitwidth")
                }
                values, err = imp.importBitsBuffer(1)
        }
diff --git a/arrow/cdata/interface.go b/arrow/cdata/interface.go
index 901371f1..f776d7f7 100644
--- a/arrow/cdata/interface.go
+++ b/arrow/cdata/interface.go
@@ -21,13 +21,13 @@ package cdata
 
 import (
        "context"
+       "errors"
        "unsafe"
 
        "github.com/apache/arrow-go/v18/arrow"
        "github.com/apache/arrow-go/v18/arrow/array"
        "github.com/apache/arrow-go/v18/arrow/arrio"
        "github.com/apache/arrow-go/v18/arrow/memory"
-       "golang.org/x/xerrors"
 )
 
 // SchemaFromPtr is a simple helper function to cast a uintptr to a 
*CArrowSchema
@@ -151,7 +151,7 @@ func ImportCRecordBatch(arr *CArrowArray, sc *CArrowSchema) 
(arrow.RecordBatch,
        }
 
        if field.Type.ID() != arrow.STRUCT {
-               return nil, xerrors.New("recordbatch array import must be of 
struct type")
+               return nil, errors.New("recordbatch array import must be of 
struct type")
        }
 
        return ImportCRecordBatchWithSchema(arr, 
arrow.NewSchema(field.Type.(*arrow.StructType).Fields(), &field.Metadata))
diff --git a/arrow/compute/utils.go b/arrow/compute/utils.go
index 7eb30705..7f7c4650 100644
--- a/arrow/compute/utils.go
+++ b/arrow/compute/utils.go
@@ -19,6 +19,7 @@
 package compute
 
 import (
+       "errors"
        "fmt"
        "io"
        "time"
@@ -30,7 +31,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/internal/debug"
        "github.com/apache/arrow-go/v18/arrow/memory"
        "github.com/apache/arrow-go/v18/internal/utils"
-       "golang.org/x/xerrors"
 )
 
 type bufferWriteSeeker struct {
@@ -84,7 +84,7 @@ func (b *bufferWriteSeeker) Seek(offset int64, whence int) 
(int64, error) {
                newpos = b.buf.Len() + offs
        }
        if newpos < 0 {
-               return 0, xerrors.New("negative result pos")
+               return 0, errors.New("negative result pos")
        }
        b.pos = newpos
        return int64(newpos), nil
diff --git a/arrow/datatype_fixedwidth.go b/arrow/datatype_fixedwidth.go
index 4dad42f9..2b24a835 100644
--- a/arrow/datatype_fixedwidth.go
+++ b/arrow/datatype_fixedwidth.go
@@ -17,7 +17,8 @@
 package arrow
 
 import (
-       "cmp"
+  "cmp"
+       "errors"        
        "fmt"
        "strconv"
        "sync"
@@ -26,7 +27,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/decimal"
        "github.com/apache/arrow-go/v18/arrow/internal/debug"
        "github.com/apache/arrow-go/v18/internal/json"
-       "golang.org/x/xerrors"
 )
 
 type BooleanType struct{}
@@ -143,11 +143,11 @@ func TimestampFromStringInLocation(val string, unit 
TimeUnit, loc *time.Location
        // more than nanosecond precision is provided
        switch {
        case unit == Second && lenWithoutZone > 19:
-               return 0, zoneFmt != "", xerrors.New("provided more than second 
precision for timestamp[s]")
+               return 0, zoneFmt != "", errors.New("provided more than second 
precision for timestamp[s]")
        case unit == Millisecond && lenWithoutZone > 23:
-               return 0, zoneFmt != "", xerrors.New("provided more than 
millisecond precision for timestamp[ms]")
+               return 0, zoneFmt != "", errors.New("provided more than 
millisecond precision for timestamp[ms]")
        case unit == Microsecond && lenWithoutZone > 26:
-               return 0, zoneFmt != "", xerrors.New("provided more than 
microsecond precision for timestamp[us]")
+               return 0, zoneFmt != "", errors.New("provided more than 
microsecond precision for timestamp[us]")
        }
 
        format += zoneFmt
@@ -220,14 +220,14 @@ func Time32FromString(val string, unit TimeUnit) (Time32, 
error) {
        switch unit {
        case Second:
                if len(val) > 8 {
-                       return 0, xerrors.New("cannot convert larger than 
second precision to time32s")
+                       return 0, errors.New("cannot convert larger than second 
precision to time32s")
                }
        case Millisecond:
                if len(val) > 12 {
-                       return 0, xerrors.New("cannot convert larger than 
millisecond precision to time32ms")
+                       return 0, errors.New("cannot convert larger than 
millisecond precision to time32ms")
                }
        case Microsecond, Nanosecond:
-               return 0, xerrors.New("time32 can only be seconds or 
milliseconds")
+               return 0, errors.New("time32 can only be seconds or 
milliseconds")
        }
 
        var (
@@ -275,10 +275,10 @@ func Time64FromString(val string, unit TimeUnit) (Time64, 
error) {
        switch unit {
        case Microsecond:
                if len(val) > 15 {
-                       return 0, xerrors.New("cannot convert larger than 
microsecond precision to time64us")
+                       return 0, errors.New("cannot convert larger than 
microsecond precision to time64us")
                }
        case Second, Millisecond:
-               return 0, xerrors.New("time64 should only be microseconds or 
nanoseconds")
+               return 0, errors.New("time64 should only be microseconds or 
nanoseconds")
        }
 
        var (
diff --git a/arrow/internal/flight_integration/scenario.go 
b/arrow/internal/flight_integration/scenario.go
index d6ac7314..b9518443 100644
--- a/arrow/internal/flight_integration/scenario.go
+++ b/arrow/internal/flight_integration/scenario.go
@@ -40,7 +40,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/internal/arrjson"
        "github.com/apache/arrow-go/v18/arrow/ipc"
        "github.com/apache/arrow-go/v18/arrow/memory"
-       "golang.org/x/xerrors"
        "google.golang.org/grpc"
        "google.golang.org/grpc/codes"
        "google.golang.org/grpc/status"
@@ -367,7 +366,7 @@ func CheckActionResults(ctx context.Context, client 
flight.Client, action *fligh
 
        res, err := stream.Recv()
        if res != nil || err != io.EOF {
-               return xerrors.New("action result stream had too many entries")
+               return errors.New("action result stream had too many entries")
        }
        return nil
 }
@@ -501,7 +500,7 @@ func (m *middlewareScenarioTester) RunClient(addr string, 
opts ...grpc.DialOptio
        // this call is expected to fail
        _, err = client.GetFlightInfo(ctx, &flight.FlightDescriptor{Type: 
flight.DescriptorCMD})
        if err == nil {
-               return xerrors.New("expected call to fail")
+               return errors.New("expected call to fail")
        }
 
        if tm.received != "expected value" {
diff --git a/arrow/scalar/nested.go b/arrow/scalar/nested.go
index aedf0a40..3a9f7aef 100644
--- a/arrow/scalar/nested.go
+++ b/arrow/scalar/nested.go
@@ -25,7 +25,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/array"
        "github.com/apache/arrow-go/v18/arrow/internal/debug"
        "github.com/apache/arrow-go/v18/arrow/memory"
-       "golang.org/x/xerrors"
 )
 
 type ListScalar interface {
@@ -331,7 +330,7 @@ func NewStructScalar(val []Scalar, typ arrow.DataType) 
*Struct {
 
 func NewStructScalarWithNames(val []Scalar, names []string) (*Struct, error) {
        if len(val) != len(names) {
-               return nil, xerrors.New("mismatching number of field names and 
child scalars")
+               return nil, errors.New("mismatching number of field names and 
child scalars")
        }
 
        fields := make([]arrow.Field, len(names))
diff --git a/arrow/scalar/scalar.go b/arrow/scalar/scalar.go
index 29885336..05146c7c 100644
--- a/arrow/scalar/scalar.go
+++ b/arrow/scalar/scalar.go
@@ -18,6 +18,7 @@ package scalar
 
 import (
        "encoding/binary"
+       "errors"
        "fmt"
        "math"
        "math/big"
@@ -37,7 +38,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/float16"
        "github.com/apache/arrow-go/v18/arrow/internal/debug"
        "github.com/apache/arrow-go/v18/arrow/memory"
-       "golang.org/x/xerrors"
 )
 
 // Scalar represents a single value of a specific DataType as opposed to
@@ -97,7 +97,7 @@ func (s *scalar) IsValid() bool { return s.Valid }
 
 func (s *scalar) Validate() error {
        if s.Type == nil {
-               return xerrors.New("scalar lacks a type")
+               return errors.New("scalar lacks a type")
        }
        return nil
 }
@@ -130,7 +130,7 @@ func (n *Null) Validate() (err error) {
                return
        }
        if n.Valid {
-               err = xerrors.New("null scalar should have Valid = false")
+               err = errors.New("null scalar should have Valid = false")
        }
        return
 }
diff --git a/go.mod b/go.mod
index 57f49fea..2f32f518 100644
--- a/go.mod
+++ b/go.mod
@@ -47,7 +47,6 @@ require (
        golang.org/x/exp v0.0.0-20260112195511-716be5621a96
        golang.org/x/sync v0.20.0
        golang.org/x/sys v0.43.0
-       golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da
        gonum.org/v1/gonum v0.17.0
        google.golang.org/grpc v1.80.0
        google.golang.org/protobuf v1.36.11
diff --git a/go.sum b/go.sum
index 6591248a..40a559f6 100644
--- a/go.sum
+++ b/go.sum
@@ -242,8 +242,6 @@ golang.org/x/tools v0.6.0/go.mod 
h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU=
 golang.org/x/tools v0.43.0 h1:12BdW9CeB3Z+J/I/wj34VMl8X+fEXBxVR90JeMX5E7s=
 golang.org/x/tools v0.43.0/go.mod 
h1:uHkMso649BX2cZK6+RpuIPXS3ho2hZo4FVwfoy1vIk0=
 golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod 
h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
-golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da 
h1:noIWHXmPHxILtqtCOPIhSt0ABwskkZKjD3bXGnZGpNY=
-golang.org/x/xerrors v0.0.0-20240903120638-7835f813f4da/go.mod 
h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90=
 gonum.org/v1/gonum v0.17.0 h1:VbpOemQlsSMrYmn7T2OUvQ4dqxQXU+ouZFQsZOx50z4=
 gonum.org/v1/gonum v0.17.0/go.mod 
h1:El3tOrEuMpv2UdMrbNlKEh9vd86bmQ6vqIcDwxEOc1E=
 google.golang.org/genproto/googleapis/rpc v0.0.0-20260120221211-b8f7ae30c516 
h1:sNrWoksmOyF5bvJUcnmbeAmQi8baNhqg5IWaI3llQqU=
diff --git a/internal/types/extension_types.go 
b/internal/types/extension_types.go
index 8257059f..eaffce65 100644
--- a/internal/types/extension_types.go
+++ b/internal/types/extension_types.go
@@ -19,12 +19,12 @@ package types
 
 import (
        "encoding/binary"
+       "errors"
        "fmt"
        "reflect"
 
        "github.com/apache/arrow-go/v18/arrow"
        "github.com/apache/arrow-go/v18/arrow/array"
-       "golang.org/x/xerrors"
 )
 
 // Parametric1Array is a simple int32 array for use with the Parametric1Type
@@ -100,7 +100,7 @@ func (Parametric1Type) Deserialize(storage arrow.DataType, 
data string) (arrow.E
        }
 
        if storage.ID() != arrow.INT32 {
-               return nil, xerrors.New("parametric1type: must have int32 as 
underlying storage type")
+               return nil, errors.New("parametric1type: must have int32 as 
underlying storage type")
        }
 
        return &Parametric1Type{arrow.ExtensionBase{Storage: 
arrow.PrimitiveTypes.Int32}, int32(binary.LittleEndian.Uint32([]byte(data)))}, 
nil
@@ -155,7 +155,7 @@ func (Parametric2Type) Deserialize(storage arrow.DataType, 
data string) (arrow.E
        }
 
        if storage.ID() != arrow.INT32 {
-               return nil, xerrors.New("parametric1type: must have int32 as 
underlying storage type")
+               return nil, errors.New("parametric1type: must have int32 as 
underlying storage type")
        }
 
        return &Parametric2Type{arrow.ExtensionBase{Storage: 
arrow.PrimitiveTypes.Int32}, int32(binary.LittleEndian.Uint32([]byte(data)))}, 
nil
@@ -214,7 +214,7 @@ func (ExtStructType) Serialize() string { return 
"ext-struct-type-unique-code" }
 // returning the correct type if it matches "ext-struct-type-unique-code".
 func (ExtStructType) Deserialize(_ arrow.DataType, serialized string) 
(arrow.ExtensionType, error) {
        if string(serialized) != "ext-struct-type-unique-code" {
-               return nil, xerrors.New("type identifier did not match")
+               return nil, errors.New("type identifier did not match")
        }
        return NewExtStructType(), nil
 }
diff --git a/parquet/file/column_reader.go b/parquet/file/column_reader.go
index 01d1e5be..48c392b9 100644
--- a/parquet/file/column_reader.go
+++ b/parquet/file/column_reader.go
@@ -28,7 +28,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet/internal/encryption"
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 const (
@@ -323,7 +322,7 @@ func (c *columnChunkReader) configureDict(page 
*DictionaryPage) error {
        }
 
        if _, ok := c.decoders[enc]; ok {
-               return xerrors.New("parquet: column chunk cannot have more than 
one dictionary.")
+               return errors.New("parquet: column chunk cannot have more than 
one dictionary")
        }
 
        switch page.Encoding() {
@@ -335,7 +334,7 @@ func (c *columnChunkReader) configureDict(page 
*DictionaryPage) error {
                decoder.SetDict(dict)
                c.decoders[enc] = decoder
        default:
-               return xerrors.New("parquet: dictionary index must be plain 
encoding")
+               return errors.New("parquet: dictionary index must be plain 
encoding")
        }
 
        // Dictionary page has been read and decoder configured
@@ -403,7 +402,7 @@ func (c *columnChunkReader) initLevelDecodersV2(page 
*DataPageV2) (int64, error)
        totalLvlLen := int64(page.repLvlByteLen) + int64(page.defLvlByteLen)
 
        if totalLvlLen > int64(len(buf)) {
-               return totalLvlLen, xerrors.New("parquet: data page too small 
for levels (corrupt header?)")
+               return totalLvlLen, errors.New("parquet: data page too small 
for levels (corrupt header?)")
        }
 
        if c.descr.MaxRepetitionLevel() > 0 {
@@ -460,7 +459,7 @@ func (c *columnChunkReader) initLevelDecodersV1(page 
*DataPageV1, repLvlEncoding
 func (c *columnChunkReader) initDataDecoder(page Page, lvlByteLen int64) error 
{
        buf := page.Data()
        if int64(len(buf)) < lvlByteLen {
-               return xerrors.New("parquet: page smaller than size of encoded 
levels")
+               return errors.New("parquet: page smaller than size of encoded 
levels")
        }
 
        buf = buf[lvlByteLen:]
@@ -566,7 +565,7 @@ func (c *columnChunkReader) determineNumToRead(batchLen 
int64, defLvls, repLvls
        if c.descr.MaxRepetitionLevel() > 0 && repLvls != nil {
                nreps := c.readRepetitionLevels(repLvls[:size])
                if defLvls != nil && ndefs != nreps {
-                       err = xerrors.New("parquet: number of decoded rep/def 
levels did not match")
+                       err = errors.New("parquet: number of decoded rep/def 
levels did not match")
                }
        }
        return
diff --git a/parquet/file/file_reader.go b/parquet/file/file_reader.go
index 4025939c..7b6df98a 100644
--- a/parquet/file/file_reader.go
+++ b/parquet/file/file_reader.go
@@ -19,6 +19,7 @@ package file
 import (
        "bytes"
        "encoding/binary"
+       "errors"
        "fmt"
        "io"
        "os"
@@ -29,7 +30,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/encryption"
        "github.com/apache/arrow-go/v18/parquet/metadata"
-       "golang.org/x/xerrors"
 )
 
 const (
@@ -39,7 +39,7 @@ const (
 var (
        magicBytes                  = []byte("PAR1")
        magicEBytes                 = []byte("PARE")
-       errInconsistentFileMetadata = xerrors.New("parquet: file is smaller 
than indicated metadata size")
+       errInconsistentFileMetadata = errors.New("parquet: file is smaller than 
indicated metadata size")
 )
 
 // Reader is the main interface for reading a parquet file
@@ -213,7 +213,7 @@ func (f *Reader) parseMetaData() error {
                }
 
                if fileDecryptProps == nil {
-                       return xerrors.New("could not read encrypted metadata, 
no decryption found in reader's properties")
+                       return errors.New("could not read encrypted metadata, 
no decryption found in reader's properties")
                }
 
                fileCryptoMetadata, err := metadata.NewFileCryptoMetaData(buf)
@@ -246,13 +246,13 @@ func (f *Reader) handleAadPrefix(fileDecrypt 
*parquet.FileDecryptionProperties,
        aadPrefixInFile := algo.Aad.AadPrefix
 
        if algo.Aad.SupplyAadPrefix && aadPrefixInProps == "" {
-               return "", xerrors.New("AAD Prefix used for file encryption but 
not stored in file and not supplied in decryption props")
+               return "", errors.New("AAD Prefix used for file encryption but 
not stored in file and not supplied in decryption props")
        }
 
        if fileHasAadPrefix {
                if aadPrefixInProps != "" {
                        if aadPrefixInProps != string(aadPrefixInFile) {
-                               return "", xerrors.New("AAD prefix in file and 
in properties but not the same")
+                               return "", errors.New("AAD prefix in file and 
in properties but not the same")
                        }
                }
                aadPrefix = aadPrefixInFile
@@ -261,10 +261,10 @@ func (f *Reader) handleAadPrefix(fileDecrypt 
*parquet.FileDecryptionProperties,
                }
        } else {
                if !algo.Aad.SupplyAadPrefix && aadPrefixInProps != "" {
-                       return "", xerrors.New("AAD Prefix set in 
decryptionproperties but was not used for file encryption")
+                       return "", errors.New("AAD Prefix set in 
decryptionproperties but was not used for file encryption")
                }
                if fileDecrypt.Verifier != nil {
-                       return "", xerrors.New("AAD Prefix Verifier is set but 
AAD Prefix not found in file")
+                       return "", errors.New("AAD Prefix Verifier is set but 
AAD Prefix not found in file")
                }
        }
        return string(append(aadPrefix, algo.Aad.AadFileUnique...)), nil
@@ -283,11 +283,11 @@ func (f *Reader) 
parseMetaDataEncryptedFilePlaintextFooter(decryptProps *parquet
                f.metadata.FileDecryptor = f.fileDecryptor
                if decryptProps.PlaintextFooterIntegrity() {
                        if len(data)-f.metadata.Size() != 
encryption.GcmTagLength+encryption.NonceLength {
-                               return xerrors.New("failed reading metadata for 
encryption signature")
+                               return errors.New("failed reading metadata for 
encryption signature")
                        }
 
                        if 
!f.metadata.VerifySignature(data[f.metadata.Size():]) {
-                               return xerrors.New("parquet crypto signature 
verification failed")
+                               return errors.New("parquet crypto signature 
verification failed")
                        }
                }
        }
diff --git a/parquet/file/file_reader_mmap.go b/parquet/file/file_reader_mmap.go
index 44e48e90..727a3879 100644
--- a/parquet/file/file_reader_mmap.go
+++ b/parquet/file/file_reader_mmap.go
@@ -20,11 +20,11 @@
 package file
 
 import (
+       "errors"
        "io"
 
        "github.com/apache/arrow-go/v18/parquet"
        "golang.org/x/exp/mmap"
-       "golang.org/x/xerrors"
 )
 
 func mmapOpen(filename string) (parquet.ReaderAtSeeker, error) {
@@ -67,10 +67,10 @@ func (m *mmapAdapter) Seek(offset int64, whence int) 
(int64, error) {
                newPos = int64(m.Len()) + offs
        }
        if newPos < 0 {
-               return 0, xerrors.New("negative result pos")
+               return 0, errors.New("negative result pos")
        }
        if newPos > int64(m.Len()) {
-               return 0, xerrors.New("new position exceeds size of file")
+               return 0, errors.New("new position exceeds size of file")
        }
        m.pos = newPos
        return newPos, nil
diff --git a/parquet/file/level_conversion.go b/parquet/file/level_conversion.go
index 76612884..5e15dd97 100644
--- a/parquet/file/level_conversion.go
+++ b/parquet/file/level_conversion.go
@@ -17,6 +17,7 @@
 package file
 
 import (
+       "errors"
        "fmt"
        "math"
        "math/bits"
@@ -27,7 +28,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet/internal/bmi"
        "github.com/apache/arrow-go/v18/parquet/internal/utils"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 type LevelInfo struct {
@@ -204,7 +204,7 @@ func DefRepLevelsToListInfo(defLevels, repLevels []int16, 
info LevelInfo, out *V
                        // offsets can be null for structs with repeated 
children
                        if offsetPos < len(offsets) {
                                if offsets[offsetPos] == math.MaxInt32 {
-                                       return xerrors.New("list index 
overflow")
+                                       return errors.New("list index overflow")
                                }
                                offsets[offsetPos]++
                        }
@@ -224,7 +224,7 @@ func DefRepLevelsToListInfo(defLevels, repLevels []int16, 
info LevelInfo, out *V
                                offsets[offsetPos] = offsets[offsetPos-1]
                                if defLevels[idx] >= info.DefLevel {
                                        if offsets[offsetPos] == math.MaxInt32 {
-                                               return xerrors.New("list index 
overflow")
+                                               return errors.New("list index 
overflow")
                                        }
                                        offsets[offsetPos]++
                                }
@@ -251,7 +251,7 @@ func DefRepLevelsToListInfo(defLevels, repLevels []int16, 
info LevelInfo, out *V
        }
 
        if out.NullCount > 0 && info.NullSlotUsage > 1 {
-               return xerrors.New("null values with null_slot_usage > 1 not 
supported.")
+               return errors.New("null values with null_slot_usage > 1 not 
supported")
        }
        return nil
 }
diff --git a/parquet/file/page_reader.go b/parquet/file/page_reader.go
index 903f2ec0..8c6c6ace 100644
--- a/parquet/file/page_reader.go
+++ b/parquet/file/page_reader.go
@@ -31,7 +31,6 @@ import (
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/thrift"
        "github.com/apache/arrow-go/v18/parquet/metadata"
-       "golang.org/x/xerrors"
 )
 
 // PageReader is the interface used by the columnreader in order to read
@@ -773,7 +772,7 @@ func (p *serializedPageReader) Next() bool {
                        p.cryptoCtx.StartDecryptWithDictionaryPage = false
                        dictHeader := p.curPageHdr.GetDictionaryPageHeader()
                        if dictHeader.GetNumValues() < 0 {
-                               p.err = xerrors.New("parquet: invalid page 
header (negative number of values)")
+                               p.err = errors.New("parquet: invalid page 
header (negative number of values)")
                                return false
                        }
 
@@ -805,7 +804,7 @@ func (p *serializedPageReader) Next() bool {
                        p.pageOrd++
                        dataHeader := p.curPageHdr.GetDataPageHeader()
                        if dataHeader.GetNumValues() < 0 {
-                               p.err = xerrors.New("parquet: invalid page 
header (negative number of values)")
+                               p.err = errors.New("parquet: invalid page 
header (negative number of values)")
                                return false
                        }
 
@@ -842,12 +841,12 @@ func (p *serializedPageReader) Next() bool {
                        p.pageOrd++
                        dataHeader := p.curPageHdr.GetDataPageHeaderV2()
                        if dataHeader.GetNumValues() < 0 {
-                               p.err = xerrors.New("parquet: invalid page 
header (negative number of values)")
+                               p.err = errors.New("parquet: invalid page 
header (negative number of values)")
                                return false
                        }
 
                        if dataHeader.GetDefinitionLevelsByteLength() < 0 || 
dataHeader.GetRepetitionLevelsByteLength() < 0 {
-                               p.err = xerrors.New("parquet: invalid page 
header (negative levels byte length)")
+                               p.err = errors.New("parquet: invalid page 
header (negative levels byte length)")
                                return false
                        }
 
@@ -860,7 +859,7 @@ func (p *serializedPageReader) Next() bool {
                        p.rowsSeen += int64(dataHeader.GetNumRows())
                        levelsBytelen, ok := 
utils.Add(int(dataHeader.GetDefinitionLevelsByteLength()), 
int(dataHeader.GetRepetitionLevelsByteLength()))
                        if !ok {
-                               p.err = xerrors.New("parquet: levels size too 
large (corrupt file?)")
+                               p.err = errors.New("parquet: levels size too 
large (corrupt file?)")
                                return false
                        }
 
diff --git a/parquet/file/page_writer.go b/parquet/file/page_writer.go
index 8d22ef83..e3b30bc7 100644
--- a/parquet/file/page_writer.go
+++ b/parquet/file/page_writer.go
@@ -18,6 +18,7 @@ package file
 
 import (
        "bytes"
+       "errors"
        "fmt"
        "math"
        "sync"
@@ -32,7 +33,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet/internal/utils"
        "github.com/apache/arrow-go/v18/parquet/metadata"
        libthrift "github.com/apache/thrift/lib/go/thrift"
-       "golang.org/x/xerrors"
 )
 
 // PageWriter is the interface for both serialized and buffered page writers
@@ -197,7 +197,7 @@ func (pw *serializedPageWriter) updateEncryption(moduleType 
int8) error {
        case encryption.DictPageModule:
                
pw.dataEncryptor.UpdateAad(encryption.CreateModuleAad(pw.dataEncryptor.FileAad(),
 moduleType, pw.rgOrdinal, pw.columnOrdinal, -1))
        default:
-               return xerrors.New("unknown module type in updateencryption")
+               return errors.New("unknown module type in updateencryption")
        }
        return nil
 }
@@ -378,7 +378,7 @@ func (pw *serializedPageWriter) WriteDataPage(page 
DataPage) (int64, error) {
                pw.setDataPageV2Header(pageHdr, dpage)
                defer dataPageV2HeaderPool.Put(pageHdr.DataPageHeaderV2)
        default:
-               return 0, xerrors.New("parquet: unexpected page type")
+               return 0, errors.New("parquet: unexpected page type")
        }
 
        startPos := pw.sink.Tell()
diff --git a/parquet/file/record_reader.go b/parquet/file/record_reader.go
index e74f17a6..40ff69d0 100644
--- a/parquet/file/record_reader.go
+++ b/parquet/file/record_reader.go
@@ -17,6 +17,7 @@
 package file
 
 import (
+       "errors"
        "fmt"
        "sync"
        "sync/atomic"
@@ -30,7 +31,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/encoding"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 // RecordReader is an interface for reading entire records/rows at a time
@@ -219,7 +219,7 @@ func (pr *primitiveRecordReader) numBytesForValues(nitems 
int64) (num int64, err
        typeSize := int64(pr.Descriptor().PhysicalType().ByteSize())
        var ok bool
        if num, ok = utils.Mul64(nitems, typeSize); !ok {
-               err = xerrors.New("total size of items too large")
+               err = errors.New("total size of items too large")
        }
        return
 }
@@ -405,14 +405,14 @@ func (rr *recordReader) LevelsPos() int64 { return 
rr.levelsPos }
 
 func updateCapacity(cap, size, extra int64) (int64, error) {
        if extra < 0 {
-               return 0, xerrors.New("negative size (corrupt file?)")
+               return 0, errors.New("negative size (corrupt file?)")
        }
        target, ok := utils.Add(size, extra)
        if !ok {
-               return 0, xerrors.New("allocation size too large (corrupt 
file?)")
+               return 0, errors.New("allocation size too large (corrupt 
file?)")
        }
        if target >= (1 << 62) {
-               return 0, xerrors.New("allocation size too large (corrupt 
file?)")
+               return 0, errors.New("allocation size too large (corrupt 
file?)")
        }
        if cap >= target {
                return cap, nil
@@ -652,7 +652,7 @@ func (rr *recordReader) ReadRecords(numRecords int64) 
(int64, error) {
                                repLevels := 
rr.RepLevels()[int(rr.levelsWritten):]
                                levelsRead, _ = 
rr.readDefinitionLevels(defLevels[:batchSize])
                                if 
rr.readRepetitionLevels(repLevels[:batchSize]) != levelsRead {
-                                       return 0, xerrors.New("number of 
decoded rep/def levels did not match")
+                                       return 0, errors.New("number of decoded 
rep/def levels did not match")
                                }
                        } else if rr.Descriptor().MaxDefinitionLevel() > 0 {
                                levelsRead, _ = 
rr.readDefinitionLevels(defLevels[:batchSize])
diff --git a/parquet/file/row_group_reader.go b/parquet/file/row_group_reader.go
index ea5f7098..295ccc1c 100644
--- a/parquet/file/row_group_reader.go
+++ b/parquet/file/row_group_reader.go
@@ -17,6 +17,7 @@
 package file
 
 import (
+       "errors"
        "fmt"
        "sync"
 
@@ -26,7 +27,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet/internal/encryption"
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/arrow-go/v18/parquet/metadata"
-       "golang.org/x/xerrors"
 )
 
 const (
@@ -134,12 +134,12 @@ func (r *RowGroupReader) GetColumnPageReader(i int) 
(PageReader, error) {
        }
 
        if r.fileDecryptor == nil {
-               return nil, xerrors.New("column in rowgroup is encrypted, but 
no file decryptor")
+               return nil, errors.New("column in rowgroup is encrypted, but no 
file decryptor")
        }
 
        const encryptedRowGroupsLimit = 32767
        if i > encryptedRowGroupsLimit {
-               return nil, xerrors.New("encrypted files cannot contain more 
than 32767 column chunks")
+               return nil, errors.New("encrypted files cannot contain more 
than 32767 column chunks")
        }
 
        if cryptoMetadata.IsSetENCRYPTION_WITH_FOOTER_KEY() {
diff --git a/parquet/file/row_group_writer.go b/parquet/file/row_group_writer.go
index b507d695..ff3361b5 100644
--- a/parquet/file/row_group_writer.go
+++ b/parquet/file/row_group_writer.go
@@ -17,11 +17,12 @@
 package file
 
 import (
+       "fmt"
+
        "github.com/apache/arrow-go/v18/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/encryption"
        "github.com/apache/arrow-go/v18/parquet/internal/utils"
        "github.com/apache/arrow-go/v18/parquet/metadata"
-       "golang.org/x/xerrors"
 )
 
 // RowGroupWriter is the base interface for writing rowgroups, the actual 
writer
@@ -117,13 +118,13 @@ func (rg *rowGroupWriter) checkRowsWritten() error {
                if rg.nrows == 0 {
                        rg.nrows = current
                } else if rg.nrows != current {
-                       return xerrors.Errorf("row mismatch for unbuffered row 
group: %d, count expected: %d, actual: %d", rg.ordinal, current, rg.nrows)
+                       return fmt.Errorf("row mismatch for unbuffered row 
group: %d, count expected: %d, actual: %d", rg.ordinal, current, rg.nrows)
                }
        } else if rg.buffered {
                current := rg.columnWriters[0].RowsWritten()
                for i, wr := range rg.columnWriters[1:] {
                        if current != wr.RowsWritten() {
-                               return xerrors.Errorf("row mismatch for 
buffered row group: %d, column: %d, count expected: %d, actual: %d", 
rg.ordinal, i+1, current, wr.RowsWritten())
+                               return fmt.Errorf("row mismatch for buffered 
row group: %d, column: %d, count expected: %d, actual: %d", rg.ordinal, i+1, 
current, wr.RowsWritten())
                        }
                }
                rg.nrows = current
@@ -205,7 +206,7 @@ func (rg *rowGroupWriter) Column(i int) (ColumnChunkWriter, 
error) {
        if i >= 0 && i < len(rg.columnWriters) {
                return rg.columnWriters[i], nil
        }
-       return nil, xerrors.Errorf("invalid column number requested: %d", i)
+       return nil, fmt.Errorf("invalid column number requested: %d", i)
 }
 
 func (rg *rowGroupWriter) CurrentColumn() int { return 
rg.metadata.CurrentColumn() }
diff --git a/parquet/internal/encoding/byte_array_decoder.go 
b/parquet/internal/encoding/byte_array_decoder.go
index f6b1027b..e568ef92 100644
--- a/parquet/internal/encoding/byte_array_decoder.go
+++ b/parquet/internal/encoding/byte_array_decoder.go
@@ -26,7 +26,6 @@ import (
        "github.com/apache/arrow-go/v18/internal/utils"
        "github.com/apache/arrow-go/v18/parquet"
        pqutils "github.com/apache/arrow-go/v18/parquet/internal/utils"
-       "golang.org/x/xerrors"
 )
 
 // PlainByteArrayDecoder decodes a data chunk for bytearrays according to
@@ -81,17 +80,17 @@ func (pbad *PlainByteArrayDecoder) Decode(out 
[]parquet.ByteArray) (int, error)
                // there should always be at least four bytes which is the 
length of the
                // next value in the data.
                if len(pbad.data) < 4 {
-                       return i, xerrors.New("parquet: eof reading bytearray")
+                       return i, errors.New("parquet: eof reading bytearray")
                }
 
                // the first 4 bytes are a little endian int32 length
                byteLen := int32(binary.LittleEndian.Uint32(pbad.data[:4]))
                if byteLen < 0 {
-                       return i, xerrors.New("parquet: invalid BYTE_ARRAY 
value")
+                       return i, errors.New("parquet: invalid BYTE_ARRAY 
value")
                }
 
                if int64(len(pbad.data)) < int64(byteLen)+4 {
-                       return i, xerrors.New("parquet: eof reading bytearray")
+                       return i, errors.New("parquet: eof reading bytearray")
                }
 
                out[i] = pbad.data[4 : byteLen+4 : byteLen+4]
@@ -111,7 +110,7 @@ func (pbad *PlainByteArrayDecoder) DecodeSpaced(out 
[]parquet.ByteArray, nullCou
                return valuesRead, err
        }
        if valuesRead != toRead {
-               return valuesRead, xerrors.New("parquet: number of values / 
definition levels read did not match")
+               return valuesRead, errors.New("parquet: number of values / 
definition levels read did not match")
        }
 
        return spacedExpand(out, nullCount, validBits, validBitsOffset), nil
diff --git a/parquet/internal/encoding/byte_stream_split.go 
b/parquet/internal/encoding/byte_stream_split.go
index 73e32634..e3199e71 100644
--- a/parquet/internal/encoding/byte_stream_split.go
+++ b/parquet/internal/encoding/byte_stream_split.go
@@ -17,6 +17,7 @@
 package encoding
 
 import (
+       "errors"
        "fmt"
        "math"
 
@@ -24,7 +25,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/memory"
        "github.com/apache/arrow-go/v18/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/debug"
-       "golang.org/x/xerrors"
 )
 
 // encodeByteStreamSplit encodes the raw bytes provided by 'in' into the 
output buffer 'data' using BYTE_STREAM_SPLIT encoding.
@@ -268,7 +268,7 @@ func (dec *ByteStreamSplitDecoder[T]) Decode(out []T) (int, 
error) {
        toRead := min(len(out), dec.nvals)
        numBytesNeeded := toRead * typeLen
        if numBytesNeeded > len(dec.data) || numBytesNeeded > math.MaxInt32 {
-               return 0, xerrors.New("parquet: eof exception")
+               return 0, errors.New("parquet: eof exception")
        }
 
        // reinterpret the output slice as bytes so that we can decode directly 
into it without an intermediate copy
@@ -297,7 +297,7 @@ func (dec *ByteStreamSplitDecoder[T]) DecodeSpaced(out []T, 
nullCount int, valid
                return valuesRead, err
        }
        if valuesRead != toRead {
-               return valuesRead, xerrors.New("parquet: number of values / 
definitions levels read did not match")
+               return valuesRead, errors.New("parquet: number of values / 
definitions levels read did not match")
        }
 
        return spacedExpand(out, nullCount, validBits, validBitsOffset), nil
diff --git a/parquet/internal/encoding/decoder.go 
b/parquet/internal/encoding/decoder.go
index 60afcc55..c45b021d 100644
--- a/parquet/internal/encoding/decoder.go
+++ b/parquet/internal/encoding/decoder.go
@@ -18,6 +18,7 @@ package encoding
 
 import (
        "bytes"
+       "errors"
 
        "github.com/apache/arrow-go/v18/arrow/array"
        "github.com/apache/arrow-go/v18/arrow/bitutil"
@@ -29,7 +30,6 @@ import (
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/utils"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 // DecoderTraits provides an interface for more easily interacting with types
@@ -133,7 +133,7 @@ func (d *dictDecoder[T]) SetData(nvals int, data []byte) 
error {
        // grab the bit width from the first byte
        width := uint8(data[0])
        if width >= 64 {
-               return xerrors.New("parquet: invalid or corrupted bit width")
+               return errors.New("parquet: invalid or corrupted bit width")
        }
 
        // pass the rest of the data, minus that first byte, to the decoder
diff --git a/parquet/internal/encoding/delta_byte_array.go 
b/parquet/internal/encoding/delta_byte_array.go
index 580c83d3..e377c9a4 100644
--- a/parquet/internal/encoding/delta_byte_array.go
+++ b/parquet/internal/encoding/delta_byte_array.go
@@ -17,10 +17,11 @@
 package encoding
 
 import (
+       "errors"
+
        "github.com/apache/arrow-go/v18/arrow/memory"
        "github.com/apache/arrow-go/v18/internal/utils"
        "github.com/apache/arrow-go/v18/parquet"
-       "golang.org/x/xerrors"
 )
 
 // DeltaByteArrayEncoder is an encoder for writing bytearrays which are delta 
encoded
@@ -270,7 +271,7 @@ func (d *DeltaByteArrayDecoder) DecodeSpaced(out 
[]parquet.ByteArray, nullCount
                return values, err
        }
        if values != toread {
-               return values, xerrors.New("parquet: number of values / 
definition levels read did not match")
+               return values, errors.New("parquet: number of values / 
definition levels read did not match")
        }
 
        return spacedExpand(out, nullCount, validBits, validBitsOffset), nil
diff --git a/parquet/internal/encoding/delta_length_byte_array.go 
b/parquet/internal/encoding/delta_length_byte_array.go
index eab064cb..a0176409 100644
--- a/parquet/internal/encoding/delta_length_byte_array.go
+++ b/parquet/internal/encoding/delta_length_byte_array.go
@@ -17,10 +17,11 @@
 package encoding
 
 import (
+       "errors"
+
        "github.com/apache/arrow-go/v18/arrow/memory"
        "github.com/apache/arrow-go/v18/internal/utils"
        "github.com/apache/arrow-go/v18/parquet"
-       "golang.org/x/xerrors"
 )
 
 // DeltaLengthByteArrayEncoder encodes data using by taking all of the byte 
array lengths
@@ -151,7 +152,7 @@ func (d *DeltaLengthByteArrayDecoder) DecodeSpaced(out 
[]parquet.ByteArray, null
        toread := len(out) - nullCount
        values, _ := d.Decode(out[:toread])
        if values != toread {
-               return values, xerrors.New("parquet: number of values / 
definition levels read did not match")
+               return values, errors.New("parquet: number of values / 
definition levels read did not match")
        }
 
        return spacedExpand(out, nullCount, validBits, validBitsOffset), nil
diff --git a/parquet/internal/encoding/fixed_len_byte_array_decoder.go 
b/parquet/internal/encoding/fixed_len_byte_array_decoder.go
index 6e389aec..8b197e7b 100644
--- a/parquet/internal/encoding/fixed_len_byte_array_decoder.go
+++ b/parquet/internal/encoding/fixed_len_byte_array_decoder.go
@@ -23,7 +23,6 @@ import (
 
        "github.com/apache/arrow-go/v18/internal/utils"
        "github.com/apache/arrow-go/v18/parquet"
-       "golang.org/x/xerrors"
 )
 
 // PlainFixedLenByteArrayDecoder is a plain encoding decoder for Fixed Length 
Byte Arrays
@@ -55,7 +54,7 @@ func (pflba *PlainFixedLenByteArrayDecoder) Decode(out 
[]parquet.FixedLenByteArr
        max := utils.Min(len(out), pflba.nvals)
        numBytesNeeded := max * pflba.typeLen
        if numBytesNeeded > len(pflba.data) || numBytesNeeded > math.MaxInt32 {
-               return 0, xerrors.New("parquet: eof exception")
+               return 0, errors.New("parquet: eof exception")
        }
 
        for idx := range out[:max] {
@@ -75,7 +74,7 @@ func (pflba *PlainFixedLenByteArrayDecoder) DecodeSpaced(out 
[]parquet.FixedLenB
                return valuesRead, err
        }
        if valuesRead != toRead {
-               return valuesRead, xerrors.New("parquet: number of values / 
definitions levels read did not match")
+               return valuesRead, errors.New("parquet: number of values / 
definitions levels read did not match")
        }
 
        return spacedExpand(out, nullCount, validBits, validBitsOffset), nil
@@ -123,7 +122,7 @@ func (dec *ByteStreamSplitFixedLenByteArrayDecoder) 
Decode(out []parquet.FixedLe
        toRead := min(len(out), dec.nvals)
        numBytesNeeded := toRead * dec.typeLen
        if numBytesNeeded > len(dec.data) || numBytesNeeded > math.MaxInt32 {
-               return 0, xerrors.New("parquet: eof exception")
+               return 0, errors.New("parquet: eof exception")
        }
 
        for i := range out {
@@ -157,7 +156,7 @@ func (dec *ByteStreamSplitFixedLenByteArrayDecoder) 
DecodeSpaced(out []parquet.F
                return valuesRead, err
        }
        if valuesRead != toRead {
-               return valuesRead, xerrors.New("parquet: number of values / 
definitions levels read did not match")
+               return valuesRead, errors.New("parquet: number of values / 
definitions levels read did not match")
        }
 
        return spacedExpand(out, nullCount, validBits, validBitsOffset), nil
diff --git a/parquet/internal/encoding/types.go 
b/parquet/internal/encoding/types.go
index ccd3db2a..a2f1a5b2 100644
--- a/parquet/internal/encoding/types.go
+++ b/parquet/internal/encoding/types.go
@@ -17,6 +17,7 @@
 package encoding
 
 import (
+       "errors"
        "io"
        "sync"
        "unsafe"
@@ -26,7 +27,6 @@ import (
        "github.com/apache/arrow-go/v18/arrow/memory"
        "github.com/apache/arrow-go/v18/internal/utils"
        "github.com/apache/arrow-go/v18/parquet"
-       "golang.org/x/xerrors"
 )
 
 // TypedDecoder is the general interface for all decoder types which can
@@ -467,7 +467,7 @@ func (b *BufferWriter) Seek(offset int64, whence int) 
(int64, error) {
                newPos = b.buffer.Len() + offs
        }
        if newPos < 0 {
-               return 0, xerrors.New("negative result pos")
+               return 0, errors.New("negative result pos")
        }
        b.pos = newPos
        return int64(newPos), nil
diff --git a/parquet/internal/utils/rle.go b/parquet/internal/utils/rle.go
index 94a40a56..23a86598 100644
--- a/parquet/internal/utils/rle.go
+++ b/parquet/internal/utils/rle.go
@@ -22,11 +22,11 @@ package utils
 import (
        "bytes"
        "encoding/binary"
+       "errors"
        "math"
 
        "github.com/apache/arrow-go/v18/arrow/bitutil"
        "github.com/apache/arrow-go/v18/internal/bitutils"
-       "golang.org/x/xerrors"
 )
 
 const (
@@ -425,7 +425,7 @@ func (r *RleEncoder) Put(value uint64) error {
        } else {
                if r.repCount >= 8 {
                        if !r.flushRepeated() {
-                               return xerrors.New("failed to flush repeated 
value")
+                               return errors.New("failed to flush repeated 
value")
                        }
                }
                r.repCount = 1
diff --git a/parquet/internal/utils/typed_rle_dict.go 
b/parquet/internal/utils/typed_rle_dict.go
index d7cb6d69..eafae2b8 100644
--- a/parquet/internal/utils/typed_rle_dict.go
+++ b/parquet/internal/utils/typed_rle_dict.go
@@ -18,10 +18,10 @@ package utils
 
 import (
        "bytes"
+       "errors"
 
        "github.com/apache/arrow-go/v18/internal/bitutils"
        "github.com/apache/arrow-go/v18/parquet"
-       "golang.org/x/xerrors"
 )
 
 func getspaced[T parquet.ColumnTypes | uint64](r *RleDecoder, dc 
DictionaryConverter[T], vals []T, batchSize, nullCount int, validBits []byte, 
validBitsOffset int64) (int, error) {
@@ -93,11 +93,11 @@ func consumeLiterals[T parquet.ColumnTypes | uint64](r 
*RleDecoder, dc Dictionar
 
        n, _ := r.r.GetBatchIndex(uint(r.bitWidth), buf)
        if n != batch {
-               return 0, 0, run, xerrors.New("was not able to retrieve correct 
number of indexes")
+               return 0, 0, run, errors.New("was not able to retrieve correct 
number of indexes")
        }
 
        if !dc.IsValid(buf...) {
-               return 0, 0, run, xerrors.New("invalid index values found for 
dictionary converter")
+               return 0, 0, run, errors.New("invalid index values found for 
dictionary converter")
        }
 
        var (
diff --git a/parquet/metadata/column_chunk.go b/parquet/metadata/column_chunk.go
index 848d20fe..e3533fb2 100644
--- a/parquet/metadata/column_chunk.go
+++ b/parquet/metadata/column_chunk.go
@@ -19,6 +19,7 @@ package metadata
 import (
        "bytes"
        "context"
+       "errors"
        "io"
        "reflect"
 
@@ -29,7 +30,6 @@ import (
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/thrift"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 // PageEncodingStats is used for counting the number of pages of specific
@@ -106,7 +106,7 @@ func NewColumnChunkMetaData(column *format.ColumnChunk, 
descr *schema.Column, wr
                                thrift.DeserializeThrift(&c.decryptedMeta, 
decryptor.Decrypt(column.GetEncryptedColumnMetadata()))
                                c.columnMeta = &c.decryptedMeta
                        } else {
-                               return nil, xerrors.New("cannot decrypt column 
metadata. file decryption not setup correctly")
+                               return nil, errors.New("cannot decrypt column 
metadata. file decryption not setup correctly")
                        }
                }
        }
diff --git a/parquet/metadata/file.go b/parquet/metadata/file.go
index 0c043f84..0b5d1284 100644
--- a/parquet/metadata/file.go
+++ b/parquet/metadata/file.go
@@ -20,6 +20,7 @@ import (
        "bytes"
        "context"
        "crypto/subtle"
+       "errors"
        "fmt"
        "io"
        "reflect"
@@ -31,7 +32,6 @@ import (
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/arrow-go/v18/parquet/internal/thrift"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 // DefaultCompressionType is used unless a different compression is specified
@@ -407,7 +407,7 @@ func (f *FileMetaData) SetFilePath(path string) {
 // current file metadata
 func (f *FileMetaData) AppendRowGroups(other *FileMetaData) error {
        if !f.Schema.Equals(other.Schema) {
-               return xerrors.New("parquet/FileMetaData: AppendRowGroups 
requires equal schemas")
+               return errors.New("parquet/FileMetaData: AppendRowGroups 
requires equal schemas")
        }
 
        f.RowGroups = append(f.RowGroups, other.GetRowGroups()...)
diff --git a/parquet/pqarrow/file_reader.go b/parquet/pqarrow/file_reader.go
index 6fba9b53..34992f8a 100644
--- a/parquet/pqarrow/file_reader.go
+++ b/parquet/pqarrow/file_reader.go
@@ -34,7 +34,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet/file"
        "github.com/apache/arrow-go/v18/parquet/schema"
        "golang.org/x/sync/errgroup"
-       "golang.org/x/xerrors"
 )
 
 type itrFactory func(int, *file.Reader) *columnIterator
@@ -509,7 +508,7 @@ func (fr *FileReader) GetRecordReader(ctx context.Context, 
colIndices, rowGroups
        }
 
        if len(readers) == 0 {
-               return nil, xerrors.New("no leaf column readers matched col 
indices")
+               return nil, errors.New("no leaf column readers matched col 
indices")
        }
 
        nrows := int64(0)
@@ -537,7 +536,7 @@ func (fr *FileReader) getReader(ctx context.Context, field 
*SchemaField, arrowFi
        rctx := readerCtxFromContext(ctx)
        if len(field.Children) == 0 {
                if !field.IsLeaf() {
-                       return nil, xerrors.New("parquet non-leaf node has no 
children")
+                       return nil, errors.New("parquet non-leaf node has no 
children")
                }
                if rctx.filterLeaves && !rctx.includesLeaf(field.ColIndex) {
                        return nil, nil
diff --git a/parquet/pqarrow/file_writer.go b/parquet/pqarrow/file_writer.go
index 6c305c59..dc2c90be 100644
--- a/parquet/pqarrow/file_writer.go
+++ b/parquet/pqarrow/file_writer.go
@@ -21,6 +21,7 @@ package pqarrow
 import (
        "context"
        "encoding/base64"
+       "errors"
        "fmt"
        "io"
 
@@ -30,7 +31,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet"
        "github.com/apache/arrow-go/v18/parquet/file"
        "github.com/apache/arrow-go/v18/parquet/metadata"
-       "golang.org/x/xerrors"
 )
 
 // WriteTable is a convenience function to create and write a full array.Table 
to a parquet file. The schema
@@ -286,7 +286,7 @@ func (fw *FileWriter) WriteTable(tbl arrow.Table, chunkSize 
int64) error {
                return fmt.Errorf("invalid write call: FileWriter is already 
closed")
        }
        if chunkSize <= 0 && tbl.NumRows() > 0 {
-               return xerrors.New("chunk size per row group must be greater 
than 0")
+               return errors.New("chunk size per row group must be greater 
than 0")
        } else if !tbl.Schema().Equal(fw.schema) {
                return fmt.Errorf("table schema does not match writer's. 
\nTable: %s\n writer: %s", tbl.Schema(), fw.schema)
        } else if chunkSize > fw.wr.Properties().MaxRowGroupLength() {
diff --git a/parquet/pqarrow/path_builder.go b/parquet/pqarrow/path_builder.go
index 63e4de43..a03e3187 100644
--- a/parquet/pqarrow/path_builder.go
+++ b/parquet/pqarrow/path_builder.go
@@ -17,6 +17,7 @@
 package pqarrow
 
 import (
+       "errors"
        "fmt"
        "sync/atomic"
        "unsafe"
@@ -27,7 +28,6 @@ import (
        "github.com/apache/arrow-go/v18/internal/bitutils"
        "github.com/apache/arrow-go/v18/internal/utils"
        "github.com/apache/arrow-go/v18/parquet/internal/encoding"
-       "golang.org/x/xerrors"
 )
 
 type iterResult int8
@@ -442,7 +442,7 @@ func (p *pathBuilder) Visit(arr arrow.Array) error {
        case arrow.EXTENSION:
                return p.Visit(arr.(array.ExtensionArray).Storage())
        case arrow.SPARSE_UNION, arrow.DENSE_UNION:
-               return xerrors.New("union types aren't supported in parquet")
+               return errors.New("union types aren't supported in parquet")
        default:
                p.addTerminalInfo(arr)
                return nil
diff --git a/parquet/pqarrow/schema.go b/parquet/pqarrow/schema.go
index aac64825..fa2246f3 100644
--- a/parquet/pqarrow/schema.go
+++ b/parquet/pqarrow/schema.go
@@ -18,6 +18,7 @@ package pqarrow
 
 import (
        "encoding/base64"
+       "errors"
        "fmt"
        "math"
        "strconv"
@@ -31,7 +32,6 @@ import (
        "github.com/apache/arrow-go/v18/parquet/file"
        "github.com/apache/arrow-go/v18/parquet/metadata"
        "github.com/apache/arrow-go/v18/parquet/schema"
-       "golang.org/x/xerrors"
 )
 
 // SchemaField is a holder that defines a specific logical field in the schema
@@ -295,7 +295,7 @@ func fieldToNode(name string, field arrow.Field, props 
*parquet.WriterProperties
        switch field.Type.ID() {
        case arrow.NULL:
                if repType != parquet.Repetitions.Optional {
-                       return nil, xerrors.New("nulltype arrow field must be 
nullable")
+                       return nil, errors.New("nulltype arrow field must be 
nullable")
                }
        case arrow.STRUCT:
                return structToNode(field, props, arrprops)
@@ -438,7 +438,7 @@ func arrowInt(log schema.IntLogicalType) (arrow.DataType, 
error) {
                }
                return arrow.PrimitiveTypes.Uint64, nil
        default:
-               return nil, xerrors.New("invalid logical type for int32")
+               return nil, errors.New("invalid logical type for int32")
        }
 }
 
@@ -447,7 +447,7 @@ func arrowTime32(logical schema.TimeLogicalType) 
(arrow.DataType, error) {
                return arrow.FixedWidthTypes.Time32ms, nil
        }
 
-       return nil, xerrors.New(logical.String() + " cannot annotate a time32")
+       return nil, errors.New(logical.String() + " cannot annotate a time32")
 }
 
 func arrowTime64(logical schema.TimeLogicalType) (arrow.DataType, error) {
@@ -457,7 +457,7 @@ func arrowTime64(logical schema.TimeLogicalType) 
(arrow.DataType, error) {
        case schema.TimeUnitNanos:
                return arrow.FixedWidthTypes.Time64ns, nil
        default:
-               return nil, xerrors.New(logical.String() + " cannot annotate 
int64")
+               return nil, errors.New(logical.String() + " cannot annotate 
int64")
        }
 }
 
@@ -478,7 +478,7 @@ func arrowTimestamp(logical schema.TimestampLogicalType) 
(arrow.DataType, error)
        case schema.TimeUnitNanos:
                return &arrow.TimestampType{TimeZone: tz, Unit: 
arrow.Nanosecond}, nil
        default:
-               return nil, xerrors.New("Unrecognized unit in timestamp logical 
type " + logical.String())
+               return nil, errors.New("Unrecognized unit in timestamp logical 
type " + logical.String())
        }
 }
 
@@ -502,7 +502,7 @@ func arrowFromInt32(logical schema.LogicalType) 
(arrow.DataType, error) {
        case schema.DateLogicalType:
                return arrow.FixedWidthTypes.Date32, nil
        default:
-               return nil, xerrors.New(logical.String() + " cannot annotate 
int32")
+               return nil, errors.New(logical.String() + " cannot annotate 
int32")
        }
 }
 
@@ -521,7 +521,7 @@ func arrowFromInt64(logical schema.LogicalType) 
(arrow.DataType, error) {
        case schema.TimestampLogicalType:
                return arrowTimestamp(logtype)
        default:
-               return nil, xerrors.New(logical.String() + " cannot annotate 
int64")
+               return nil, errors.New(logical.String() + " cannot annotate 
int64")
        }
 }
 
@@ -537,7 +537,7 @@ func arrowFromByteArray(logical schema.LogicalType) 
(arrow.DataType, error) {
                schema.BSONLogicalType:
                return arrow.BinaryTypes.Binary, nil
        default:
-               return nil, xerrors.New("unhandled logicaltype " + 
logical.String() + " for byte_array")
+               return nil, errors.New("unhandled logicaltype " + 
logical.String() + " for byte_array")
        }
 }
 
@@ -556,7 +556,7 @@ func arrowFromFLBA(logical schema.LogicalType, length int) 
(arrow.DataType, erro
        case schema.Float16LogicalType:
                return &arrow.Float16Type{}, nil
        default:
-               return nil, xerrors.New("unhandled logical type " + 
logical.String() + " for fixed-length byte array")
+               return nil, errors.New("unhandled logical type " + 
logical.String() + " for fixed-length byte array")
        }
 }
 
@@ -662,7 +662,7 @@ func getArrowType(physical parquet.Type, logical 
schema.LogicalType, typeLen int
        case parquet.Types.FixedLenByteArray:
                return arrowFromFLBA(logical, typeLen)
        default:
-               return nil, xerrors.New("invalid physical column type")
+               return nil, errors.New("invalid physical column type")
        }
 }
 
@@ -676,11 +676,11 @@ func populateLeaf(colIndex int, field *arrow.Field, 
currentLevels file.LevelInfo
 
 func listToSchemaField(n *schema.GroupNode, currentLevels file.LevelInfo, ctx 
*schemaTree, parent, out *SchemaField) error {
        if n.NumFields() != 1 {
-               return xerrors.New("LIST groups must have only 1 child")
+               return errors.New("LIST groups must have only 1 child")
        }
 
        if n.RepetitionType() == parquet.Repetitions.Repeated {
-               return xerrors.New("LIST groups must not be repeated")
+               return errors.New("LIST groups must not be repeated")
        }
 
        currentLevels.Increment(n)
@@ -691,7 +691,7 @@ func listToSchemaField(n *schema.GroupNode, currentLevels 
file.LevelInfo, ctx *s
 
        listNode := n.Field(0)
        if listNode.RepetitionType() != parquet.Repetitions.Repeated {
-               return xerrors.New("non-repeated nodes in a list group are not 
supported")
+               return errors.New("non-repeated nodes in a list group are not 
supported")
        }
 
        repeatedAncestorDef := currentLevels.IncrementRepeated()
@@ -791,29 +791,29 @@ func groupToStructField(n *schema.GroupNode, 
currentLevels file.LevelInfo, ctx *
 
 func mapToSchemaField(n *schema.GroupNode, currentLevels file.LevelInfo, ctx 
*schemaTree, parent, out *SchemaField) error {
        if n.NumFields() != 1 {
-               return xerrors.New("MAP group must have exactly 1 child")
+               return errors.New("MAP group must have exactly 1 child")
        }
        if n.RepetitionType() == parquet.Repetitions.Repeated {
-               return xerrors.New("MAP groups must not be repeated")
+               return errors.New("MAP groups must not be repeated")
        }
 
        keyvalueNode := n.Field(0)
        if keyvalueNode.RepetitionType() != parquet.Repetitions.Repeated {
-               return xerrors.New("Non-repeated keyvalue group in MAP group is 
not supported")
+               return errors.New("non-repeated keyvalue group in MAP group is 
not supported")
        }
 
        if keyvalueNode.Type() != schema.Group {
-               return xerrors.New("keyvalue node must be a group")
+               return errors.New("keyvalue node must be a group")
        }
 
        kvgroup := keyvalueNode.(*schema.GroupNode)
        if kvgroup.NumFields() != 1 && kvgroup.NumFields() != 2 {
-               return fmt.Errorf("keyvalue node group must have exactly 1 or 2 
child elements, Found %d", kvgroup.NumFields())
+               return fmt.Errorf("keyvalue node group must have exactly 1 or 2 
child elements, found %d", kvgroup.NumFields())
        }
 
        keyNode := kvgroup.Field(0)
        if keyNode.RepetitionType() != parquet.Repetitions.Required {
-               return xerrors.New("MAP keys must be required")
+               return errors.New("MAP keys must be required")
        }
 
        // Arrow doesn't support 1 column maps (i.e. Sets).  The options are to 
either
@@ -1092,7 +1092,7 @@ func applyOriginalStorageMetadata(origin arrow.Field, 
inferred *SchemaField) (mo
                        inferred.Field.Type = extType
                }
        case arrow.SPARSE_UNION, arrow.DENSE_UNION:
-               err = xerrors.New("unimplemented type")
+               err = errors.New("unimplemented type")
        case arrow.STRUCT:
                typ := origin.Type.(*arrow.StructType)
                if nchildren != typ.NumFields() {
diff --git a/parquet/schema/helpers.go b/parquet/schema/helpers.go
index 0ba6e0d6..99598018 100644
--- a/parquet/schema/helpers.go
+++ b/parquet/schema/helpers.go
@@ -17,8 +17,10 @@
 package schema
 
 import (
+       "errors"
+       "fmt"
+
        "github.com/apache/arrow-go/v18/parquet"
-       "golang.org/x/xerrors"
 )
 
 // ListOf is a convenience helper function to create a properly structured
@@ -49,11 +51,11 @@ func ListOf(n Node, rep parquet.Repetition, fieldID int32) 
(*GroupNode, error) {
 // <element-repetition> can only be optional or required.
 func ListOfWithName(listName string, element Node, rep parquet.Repetition, 
fieldID int32) (*GroupNode, error) {
        if rep == parquet.Repetitions.Repeated {
-               return nil, xerrors.Errorf("parquet: listof repetition must not 
be repeated, got :%s", rep)
+               return nil, fmt.Errorf("parquet: listof repetition must not be 
repeated, got :%s", rep)
        }
 
        if element.RepetitionType() == parquet.Repetitions.Repeated {
-               return nil, xerrors.Errorf("parquet: element repetition must 
not be repeated, got: %s", element.RepetitionType())
+               return nil, fmt.Errorf("parquet: element repetition must not be 
repeated, got: %s", element.RepetitionType())
        }
 
        switch n := element.(type) {
@@ -90,16 +92,16 @@ func ListOfWithName(listName string, element Node, rep 
parquet.Repetition, field
 // value node can be nil (omitted) or have a repetition of required or 
optional *only*.
 func MapOf(name string, key Node, value Node, mapRep parquet.Repetition, 
fieldID int32) (*GroupNode, error) {
        if mapRep == parquet.Repetitions.Repeated {
-               return nil, xerrors.Errorf("parquet: map repetition cannot be 
Repeated, got: %s", mapRep)
+               return nil, fmt.Errorf("parquet: map repetition cannot be 
Repeated, got: %s", mapRep)
        }
 
        if key.RepetitionType() != parquet.Repetitions.Required {
-               return nil, xerrors.Errorf("parquet: map key repetition must be 
Required, got: %s", key.RepetitionType())
+               return nil, fmt.Errorf("parquet: map key repetition must be 
Required, got: %s", key.RepetitionType())
        }
 
        if value != nil {
                if value.RepetitionType() == parquet.Repetitions.Repeated {
-                       return nil, xerrors.New("parquet: map value cannot have 
repetition Repeated")
+                       return nil, errors.New("parquet: map value cannot have 
repetition Repeated")
                }
                switch value := value.(type) {
                case *PrimitiveNode:
diff --git a/parquet/schema/node.go b/parquet/schema/node.go
index 10bd9b09..930c9a88 100644
--- a/parquet/schema/node.go
+++ b/parquet/schema/node.go
@@ -17,12 +17,12 @@
 package schema
 
 import (
+       "errors"
        "fmt"
 
        "github.com/apache/arrow-go/v18/parquet"
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
        "github.com/apache/thrift/lib/go/thrift"
-       "golang.org/x/xerrors"
 )
 
 // NodeType describes whether the Node is a Primitive or Group node
@@ -177,7 +177,7 @@ func NewPrimitiveNodeLogical(name string, repetition 
parquet.Repetition, logical
        }
 
        if n.physicalType == parquet.Types.FixedLenByteArray && n.typeLen <= 0 {
-               return nil, xerrors.New("invalid fixed length byte array 
length")
+               return nil, errors.New("invalid fixed length byte array length")
        }
        return n, nil
 }
@@ -201,7 +201,7 @@ func NewPrimitiveNodeConverted(name string, repetition 
parquet.Repetition, typ p
                switch typ {
                case parquet.Types.Int32, parquet.Types.Int64, 
parquet.Types.ByteArray, parquet.Types.FixedLenByteArray:
                default:
-                       return nil, xerrors.New("parquet: DECIMAL can only 
annotate INT32, INT64, BYTE_ARRAY and FIXED")
+                       return nil, errors.New("parquet: DECIMAL can only 
annotate INT32, INT64, BYTE_ARRAY and FIXED")
                }
 
                switch {
@@ -236,11 +236,11 @@ func NewPrimitiveNodeConverted(name string, repetition 
parquet.Repetition, typ p
                }
        case ConvertedTypes.Interval:
                if typ != parquet.Types.FixedLenByteArray || typeLen != 12 {
-                       return nil, xerrors.New("parquet: INTERVAL can only 
annotate FIXED_LEN_BYTE_ARRAY(12)")
+                       return nil, errors.New("parquet: INTERVAL can only 
annotate FIXED_LEN_BYTE_ARRAY(12)")
                }
        case ConvertedTypes.Enum:
                if typ != parquet.Types.ByteArray {
-                       return nil, xerrors.New("parquet: ENUM can only 
annotate BYTE_ARRAY fields")
+                       return nil, errors.New("parquet: ENUM can only annotate 
BYTE_ARRAY fields")
                }
        case ConvertedTypes.NA:
        default:
@@ -254,7 +254,7 @@ func NewPrimitiveNodeConverted(name string, repetition 
parquet.Repetition, typ p
 
        if n.physicalType == parquet.Types.FixedLenByteArray {
                if typeLen <= 0 {
-                       return nil, xerrors.New("invalid fixed len byte array 
length")
+                       return nil, errors.New("invalid fixed len byte array 
length")
                }
                n.typeLen = typeLen
        }
diff --git a/parquet/schema/schema.go b/parquet/schema/schema.go
index 3ff37689..8972dfaf 100644
--- a/parquet/schema/schema.go
+++ b/parquet/schema/schema.go
@@ -31,6 +31,7 @@
 package schema
 
 import (
+       "errors"
        "fmt"
        "io"
        "iter"
@@ -39,7 +40,6 @@ import (
 
        "github.com/apache/arrow-go/v18/parquet"
        format "github.com/apache/arrow-go/v18/parquet/internal/gen-go/parquet"
-       "golang.org/x/xerrors"
 )
 
 // Schema is the container for the converted Parquet schema with a computed
@@ -64,12 +64,12 @@ type Schema struct {
 // FromParquet converts a slice of thrift Schema Elements to the correct node 
type
 func FromParquet(elems []*format.SchemaElement) (Node, error) {
        if len(elems) == 0 {
-               return nil, xerrors.New("parquet: empty schema (no root)")
+               return nil, errors.New("parquet: empty schema (no root)")
        }
 
        if elems[0].GetNumChildren() == 0 {
                if len(elems) > 1 {
-                       return nil, xerrors.New("parquet: schema had multiple 
nodes but root had no children")
+                       return nil, errors.New("parquet: schema had multiple 
nodes but root had no children")
                }
                // parquet file with no columns
                return GroupNodeFromThrift(elems[0], []Node{})
@@ -84,7 +84,7 @@ func FromParquet(elems []*format.SchemaElement) (Node, error) 
{
 
        nextNode = func() (Node, error) {
                if pos == len(elems) {
-                       return nil, xerrors.New("parquet: malformed schema: not 
enough elements")
+                       return nil, errors.New("parquet: malformed schema: not 
enough elements")
                }
 
                elem := elems[pos]
@@ -208,7 +208,7 @@ func (s *Schema) HasRepeatedFields() bool {
 // and is used to update the schema metadata Column Orders. len(orders) must 
equal s.NumColumns()
 func (s *Schema) UpdateColumnOrders(orders []parquet.ColumnOrder) error {
        if len(orders) != s.NumColumns() {
-               return xerrors.New("parquet: malformed schema: not enough 
ColumnOrder values")
+               return errors.New("parquet: malformed schema: not enough 
ColumnOrder values")
        }
 
        visitor := schemaColumnOrderUpdater{orders, 0}

Reply via email to