...

Source file src/github.com/apache/arrow/go/v15/parquet/internal/testutils/random.go

Documentation: github.com/apache/arrow/go/v15/parquet/internal/testutils

     1  // Licensed to the Apache Software Foundation (ASF) under one
     2  // or more contributor license agreements.  See the NOTICE file
     3  // distributed with this work for additional information
     4  // regarding copyright ownership.  The ASF licenses this file
     5  // to you under the Apache License, Version 2.0 (the
     6  // "License"); you may not use this file except in compliance
     7  // with the License.  You may obtain a copy of the License at
     8  //
     9  // http://www.apache.org/licenses/LICENSE-2.0
    10  //
    11  // Unless required by applicable law or agreed to in writing, software
    12  // distributed under the License is distributed on an "AS IS" BASIS,
    13  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    14  // See the License for the specific language governing permissions and
    15  // limitations under the License.
    16  
    17  // Package testutils contains utilities for generating random data and other
    18  // helpers that are used for testing the various aspects of the parquet library.
    19  package testutils
    20  
    21  import (
    22  	"encoding/binary"
    23  	"math"
    24  	"time"
    25  	"unsafe"
    26  
    27  	"github.com/apache/arrow/go/v15/arrow"
    28  	"github.com/apache/arrow/go/v15/arrow/array"
    29  	"github.com/apache/arrow/go/v15/arrow/bitutil"
    30  	"github.com/apache/arrow/go/v15/arrow/endian"
    31  	"github.com/apache/arrow/go/v15/arrow/float16"
    32  	"github.com/apache/arrow/go/v15/arrow/memory"
    33  	"github.com/apache/arrow/go/v15/parquet"
    34  	"github.com/apache/arrow/go/v15/parquet/pqarrow"
    35  
    36  	"golang.org/x/exp/rand"
    37  	"gonum.org/v1/gonum/stat/distuv"
    38  )
    39  
    40  // RandomArrayGenerator is a struct used for constructing Random Arrow arrays
    41  // for use with testing.
    42  type RandomArrayGenerator struct {
    43  	seed     uint64
    44  	extra    uint64
    45  	src      rand.Source
    46  	seedRand *rand.Rand
    47  }
    48  
    49  // NewRandomArrayGenerator constructs a new generator with the requested Seed
    50  func NewRandomArrayGenerator(seed uint64) RandomArrayGenerator {
    51  	src := rand.NewSource(seed)
    52  	return RandomArrayGenerator{seed, 0, src, rand.New(src)}
    53  }
    54  
    55  // GenerateBitmap generates a bitmap of n bits and stores it into buffer. Prob is the probability
    56  // that a given bit will be zero, with 1-prob being the probability it will be 1. The return value
    57  // is the number of bits that were left unset. The assumption being that buffer is currently
    58  // zero initialized as this function does not clear any bits, it only sets 1s.
    59  func (r *RandomArrayGenerator) GenerateBitmap(buffer []byte, n int64, prob float64) int64 {
    60  	count := int64(0)
    61  	r.extra++
    62  
    63  	// bernoulli distribution uses P to determine the probability of a 0 or a 1,
    64  	// which we'll use to generate the bitmap.
    65  	dist := distuv.Bernoulli{P: prob, Src: rand.NewSource(r.seed + r.extra)}
    66  	for i := 0; int64(i) < n; i++ {
    67  		if dist.Rand() != float64(0.0) {
    68  			bitutil.SetBit(buffer, i)
    69  		} else {
    70  			count++
    71  		}
    72  	}
    73  
    74  	return count
    75  }
    76  
    77  // ByteArray creates an array.String for use of creating random ByteArray values for testing parquet
    78  // writing/reading. minLen/maxLen are the min and max length for a given value in the resulting array,
    79  // with nullProb being the probability of a given index being null.
    80  //
    81  // For this generation we only generate ascii values with a min of 'A' and max of 'z'.
    82  func (r *RandomArrayGenerator) ByteArray(size int64, minLen, maxLen int32, nullProb float64) arrow.Array {
    83  	if nullProb < 0 || nullProb > 1 {
    84  		panic("null prob must be between 0 and 1")
    85  	}
    86  
    87  	lengths := r.Int32(size, minLen, maxLen, nullProb)
    88  	defer lengths.Release()
    89  
    90  	r.extra++
    91  	dist := rand.New(rand.NewSource(r.seed + r.extra))
    92  	bldr := array.NewStringBuilder(memory.DefaultAllocator)
    93  	defer bldr.Release()
    94  
    95  	strbuf := make([]byte, maxLen)
    96  
    97  	for i := 0; int64(i) < size; i++ {
    98  		if lengths.IsValid(i) {
    99  			l := lengths.Value(i)
   100  			for j := int32(0); j < l; j++ {
   101  				strbuf[j] = byte(dist.Int31n(int32('z')-int32('A')+1) + int32('A'))
   102  			}
   103  			val := strbuf[:l]
   104  			bldr.Append(*(*string)(unsafe.Pointer(&val)))
   105  		} else {
   106  			bldr.AppendNull()
   107  		}
   108  	}
   109  
   110  	return bldr.NewArray()
   111  }
   112  
   113  // Uint8 generates a random array.Uint8 of the requested size whose values are between min and max
   114  // with prob as the probability that a given index will be null.
   115  func (r *RandomArrayGenerator) Uint8(size int64, min, max uint8, prob float64) arrow.Array {
   116  	buffers := make([]*memory.Buffer, 2)
   117  	nullCount := int64(0)
   118  
   119  	buffers[0] = memory.NewResizableBuffer(memory.DefaultAllocator)
   120  	buffers[0].Resize(int(bitutil.BytesForBits(size)))
   121  	nullCount = r.GenerateBitmap(buffers[0].Bytes(), size, prob)
   122  
   123  	buffers[1] = memory.NewResizableBuffer(memory.DefaultAllocator)
   124  	buffers[1].Resize(int(size * int64(arrow.Uint8SizeBytes)))
   125  
   126  	r.extra++
   127  	dist := rand.New(rand.NewSource(r.seed + r.extra))
   128  	out := arrow.Uint8Traits.CastFromBytes(buffers[1].Bytes())
   129  	for i := int64(0); i < size; i++ {
   130  		out[i] = uint8(dist.Intn(int(max-min+1))) + min
   131  	}
   132  
   133  	return array.NewUint8Data(array.NewData(arrow.PrimitiveTypes.Uint8, int(size), buffers, nil, int(nullCount), 0))
   134  }
   135  
   136  // Int32 generates a random array.Int32 of the given size with each value between min and max,
   137  // and pctNull as the probability that a given index will be null.
   138  func (r *RandomArrayGenerator) Int32(size int64, min, max int32, pctNull float64) *array.Int32 {
   139  	buffers := make([]*memory.Buffer, 2)
   140  	nullCount := int64(0)
   141  
   142  	buffers[0] = memory.NewResizableBuffer(memory.DefaultAllocator)
   143  	buffers[0].Resize(int(bitutil.BytesForBits(size)))
   144  	nullCount = r.GenerateBitmap(buffers[0].Bytes(), size, 1-pctNull)
   145  
   146  	buffers[1] = memory.NewResizableBuffer(memory.DefaultAllocator)
   147  	buffers[1].Resize(arrow.Int32Traits.BytesRequired(int(size)))
   148  
   149  	r.extra++
   150  	dist := rand.New(rand.NewSource(r.seed + r.extra))
   151  	out := arrow.Int32Traits.CastFromBytes(buffers[1].Bytes())
   152  	for i := int64(0); i < size; i++ {
   153  		out[i] = dist.Int31n(max-min+1) + min
   154  	}
   155  	return array.NewInt32Data(array.NewData(arrow.PrimitiveTypes.Int32, int(size), buffers, nil, int(nullCount), 0))
   156  }
   157  
   158  // Int64 generates a random array.Int64 of the given size with each value between min and max,
   159  // and pctNull as the probability that a given index will be null.
   160  func (r *RandomArrayGenerator) Int64(size int64, min, max int64, pctNull float64) *array.Int64 {
   161  	buffers := make([]*memory.Buffer, 2)
   162  	nullCount := int64(0)
   163  
   164  	buffers[0] = memory.NewResizableBuffer(memory.DefaultAllocator)
   165  	buffers[0].Resize(int(bitutil.BytesForBits(size)))
   166  	nullCount = r.GenerateBitmap(buffers[0].Bytes(), size, 1-pctNull)
   167  
   168  	buffers[1] = memory.NewResizableBuffer(memory.DefaultAllocator)
   169  	buffers[1].Resize(arrow.Int64Traits.BytesRequired(int(size)))
   170  
   171  	r.extra++
   172  	dist := rand.New(rand.NewSource(r.seed + r.extra))
   173  	out := arrow.Int64Traits.CastFromBytes(buffers[1].Bytes())
   174  	for i := int64(0); i < size; i++ {
   175  		out[i] = dist.Int63n(max-min+1) + min
   176  	}
   177  	return array.NewInt64Data(array.NewData(arrow.PrimitiveTypes.Int64, int(size), buffers, nil, int(nullCount), 0))
   178  }
   179  
   180  // Float64 generates a random array.Float64 of the requested size with pctNull as the probability
   181  // that a given index will be null.
   182  func (r *RandomArrayGenerator) Float64(size int64, pctNull float64) *array.Float64 {
   183  	buffers := make([]*memory.Buffer, 2)
   184  	nullCount := int64(0)
   185  
   186  	buffers[0] = memory.NewResizableBuffer(memory.DefaultAllocator)
   187  	buffers[0].Resize(int(bitutil.BytesForBits(size)))
   188  	nullCount = r.GenerateBitmap(buffers[0].Bytes(), size, 1-pctNull)
   189  
   190  	buffers[1] = memory.NewResizableBuffer(memory.DefaultAllocator)
   191  	buffers[1].Resize(arrow.Float64Traits.BytesRequired(int(size)))
   192  
   193  	r.extra++
   194  	dist := rand.New(rand.NewSource(r.seed + r.extra))
   195  	out := arrow.Float64Traits.CastFromBytes(buffers[1].Bytes())
   196  	for i := int64(0); i < size; i++ {
   197  		out[i] = dist.NormFloat64()
   198  	}
   199  	return array.NewFloat64Data(array.NewData(arrow.PrimitiveTypes.Float64, int(size), buffers, nil, int(nullCount), 0))
   200  }
   201  
   202  func (r *RandomArrayGenerator) StringWithRepeats(mem memory.Allocator, sz, unique int64, minLen, maxLen int32, nullProb float64) *array.String {
   203  	if unique > sz {
   204  		panic("invalid config for random StringWithRepeats")
   205  	}
   206  
   207  	// generate a random string dictionary without any nulls
   208  	arr := r.ByteArray(unique, minLen, maxLen, 0)
   209  	defer arr.Release()
   210  	dict := arr.(*array.String)
   211  
   212  	// generate random indices to sample dictionary with
   213  	idArray := r.Int64(sz, 0, unique-1, nullProb)
   214  	defer idArray.Release()
   215  
   216  	bldr := array.NewStringBuilder(mem)
   217  	defer bldr.Release()
   218  
   219  	for i := int64(0); i < sz; i++ {
   220  		if idArray.IsValid(int(i)) {
   221  			idx := idArray.Value(int(i))
   222  			bldr.Append(dict.Value(int(idx)))
   223  		} else {
   224  			bldr.AppendNull()
   225  		}
   226  	}
   227  
   228  	return bldr.NewStringArray()
   229  }
   230  
   231  // FillRandomInt8 populates the slice out with random int8 values between min and max using
   232  // seed as the random see for generation to allow consistency for testing.
   233  func FillRandomInt8(seed uint64, min, max int8, out []int8) {
   234  	r := rand.New(rand.NewSource(seed))
   235  	for idx := range out {
   236  		out[idx] = int8(r.Intn(int(max-min+1))) + min
   237  	}
   238  }
   239  
   240  // FillRandomUint8 populates the slice out with random uint8 values between min and max using
   241  // seed as the random see for generation to allow consistency for testing.
   242  func FillRandomUint8(seed uint64, min, max uint8, out []uint8) {
   243  	r := rand.New(rand.NewSource(seed))
   244  	for idx := range out {
   245  		out[idx] = uint8(r.Intn(int(max-min+1))) + min
   246  	}
   247  }
   248  
   249  // FillRandomInt16 populates the slice out with random int16 values between min and max using
   250  // seed as the random see for generation to allow consistency for testing.
   251  func FillRandomInt16(seed uint64, min, max int16, out []int16) {
   252  	r := rand.New(rand.NewSource(seed))
   253  	for idx := range out {
   254  		out[idx] = int16(r.Intn(int(max-min+1))) + min
   255  	}
   256  }
   257  
   258  // FillRandomUint16 populates the slice out with random uint16 values between min and max using
   259  // seed as the random see for generation to allow consistency for testing.
   260  func FillRandomUint16(seed uint64, min, max uint16, out []uint16) {
   261  	r := rand.New(rand.NewSource(seed))
   262  	for idx := range out {
   263  		out[idx] = uint16(r.Intn(int(max-min+1))) + min
   264  	}
   265  }
   266  
   267  // FillRandomInt32 populates out with random int32 values using seed as the random
   268  // seed for the generator to allow consistency for testing.
   269  func FillRandomInt32(seed uint64, out []int32) {
   270  	r := rand.New(rand.NewSource(seed))
   271  	for idx := range out {
   272  		out[idx] = int32(r.Uint32())
   273  	}
   274  }
   275  
   276  // FillRandomInt32Max populates out with random int32 values between 0 and max using seed as the random
   277  // seed for the generator to allow consistency for testing.
   278  func FillRandomInt32Max(seed uint64, max int32, out []int32) {
   279  	r := rand.New(rand.NewSource(seed))
   280  	for idx := range out {
   281  		out[idx] = r.Int31n(max)
   282  	}
   283  }
   284  
   285  // FillRandomUint32Max populates out with random uint32 values between 0 and max using seed as the random
   286  // seed for the generator to allow consistency for testing.
   287  func FillRandomUint32Max(seed uint64, max uint32, out []uint32) {
   288  	r := rand.New(rand.NewSource(seed))
   289  	for idx := range out {
   290  		out[idx] = uint32(r.Uint64n(uint64(max)))
   291  	}
   292  }
   293  
   294  // FillRandomInt64Max populates out with random int64 values between 0 and max using seed as the random
   295  // seed for the generator to allow consistency for testing.
   296  func FillRandomInt64Max(seed uint64, max int64, out []int64) {
   297  	r := rand.New(rand.NewSource(seed))
   298  	for idx := range out {
   299  		out[idx] = r.Int63n(max)
   300  	}
   301  }
   302  
   303  // FillRandomUint32 populates out with random uint32 values using seed as the random
   304  // seed for the generator to allow consistency for testing.
   305  func FillRandomUint32(seed uint64, out []uint32) {
   306  	r := rand.New(rand.NewSource(seed))
   307  	for idx := range out {
   308  		out[idx] = r.Uint32()
   309  	}
   310  }
   311  
   312  // FillRandomUint64 populates out with random uint64 values using seed as the random
   313  // seed for the generator to allow consistency for testing.
   314  func FillRandomUint64(seed uint64, out []uint64) {
   315  	r := rand.New(rand.NewSource(seed))
   316  	for idx := range out {
   317  		out[idx] = r.Uint64()
   318  	}
   319  }
   320  
   321  // FillRandomUint64Max populates out with random uint64 values between 0 and max using seed as the random
   322  // seed for the generator to allow consistency for testing.
   323  func FillRandomUint64Max(seed uint64, max uint64, out []uint64) {
   324  	r := rand.New(rand.NewSource(seed))
   325  	for idx := range out {
   326  		out[idx] = r.Uint64n(max)
   327  	}
   328  }
   329  
   330  // FillRandomInt64 populates out with random int64 values using seed as the random
   331  // seed for the generator to allow consistency for testing.
   332  func FillRandomInt64(seed uint64, out []int64) {
   333  	r := rand.New(rand.NewSource(seed))
   334  	for idx := range out {
   335  		out[idx] = int64(r.Uint64())
   336  	}
   337  }
   338  
   339  // FillRandomInt96 populates out with random Int96 values using seed as the random
   340  // seed for the generator to allow consistency for testing. It does this by generating
   341  // three random uint32 values for each int96 value.
   342  func FillRandomInt96(seed uint64, out []parquet.Int96) {
   343  	r := rand.New(rand.NewSource(seed))
   344  	for idx := range out {
   345  		*(*int32)(unsafe.Pointer(&out[idx][0])) = int32(r.Uint32())
   346  		*(*int32)(unsafe.Pointer(&out[idx][4])) = int32(r.Uint32())
   347  		*(*int32)(unsafe.Pointer(&out[idx][8])) = int32(r.Uint32())
   348  	}
   349  }
   350  
   351  // randFloat32 creates a random float value with a normal distribution
   352  // to better spread the values out and ensure we do not return any NaN values.
   353  func randFloat32(r *rand.Rand) float32 {
   354  	for {
   355  		f := math.Float32frombits(r.Uint32())
   356  		if !math.IsNaN(float64(f)) {
   357  			return f
   358  		}
   359  	}
   360  }
   361  
   362  // randFloat64 creates a random float value with a normal distribution
   363  // to better spread the values out and ensure we do not return any NaN values.
   364  func randFloat64(r *rand.Rand) float64 {
   365  	for {
   366  		f := math.Float64frombits(r.Uint64())
   367  		if !math.IsNaN(f) {
   368  			return f
   369  		}
   370  	}
   371  }
   372  
   373  // randFloat16 creates a random float value with a normal distribution
   374  // to better spread the values out and ensure we do not return any NaN or Inf values.
   375  func randFloat16(r *rand.Rand) float16.Num {
   376  	for {
   377  		f := float16.FromBits(uint16(r.Uint64n(math.MaxUint16 + 1)))
   378  		if !f.IsNaN() {
   379  			return f
   380  		}
   381  	}
   382  }
   383  
   384  // FillRandomFloat32 populates out with random float32 values using seed as the random
   385  // seed for the generator to allow consistency for testing.
   386  func FillRandomFloat32(seed uint64, out []float32) {
   387  	r := rand.New(rand.NewSource(seed))
   388  	for idx := range out {
   389  		out[idx] = randFloat32(r)
   390  	}
   391  }
   392  
   393  // FillRandomFloat64 populates out with random float64 values using seed as the random
   394  // seed for the generator to allow consistency for testing.
   395  func FillRandomFloat64(seed uint64, out []float64) {
   396  	r := rand.New(rand.NewSource(seed))
   397  	for idx := range out {
   398  		out[idx] = randFloat64(r)
   399  	}
   400  }
   401  
   402  // FillRandomFloat16 populates out with random float64 values using seed as the random
   403  // seed for the generator to allow consistency for testing.
   404  func FillRandomFloat16(seed uint64, out []float16.Num) {
   405  	r := rand.New(rand.NewSource(seed))
   406  	for idx := range out {
   407  		out[idx] = randFloat16(r)
   408  	}
   409  }
   410  
   411  // FillRandomByteArray populates out with random ByteArray values with lengths between 2 and 12
   412  // using heap as the actual memory storage used for the bytes generated. Each element of
   413  // out will be some slice of the bytes in heap, and as such heap must outlive the byte array slices.
   414  func FillRandomByteArray(seed uint64, out []parquet.ByteArray, heap *memory.Buffer) {
   415  	const (
   416  		maxByteArrayLen = 12
   417  		minByteArrayLen = 2
   418  	)
   419  	RandomByteArray(seed, out, heap, minByteArrayLen, maxByteArrayLen)
   420  }
   421  
   422  // FillRandomFixedByteArray populates out with random FixedLenByteArray values with of a length equal to size
   423  // using heap as the actual memory storage used for the bytes generated. Each element of
   424  // out will be a slice of size bytes in heap, and as such heap must outlive the byte array slices.
   425  func FillRandomFixedByteArray(seed uint64, out []parquet.FixedLenByteArray, heap *memory.Buffer, size int) {
   426  	heap.Resize(len(out) * size)
   427  
   428  	buf := heap.Bytes()
   429  	r := rand.New(rand.NewSource(seed))
   430  	for idx := range out {
   431  		r.Read(buf[:size])
   432  		out[idx] = buf[:size]
   433  		buf = buf[size:]
   434  	}
   435  }
   436  
   437  // FillRandomBooleans populates out with random bools with the probability p of being false using
   438  // seed as the random seed to the generator in order to allow consistency for testing. This uses
   439  // a Bernoulli distribution of values.
   440  func FillRandomBooleans(p float64, seed uint64, out []bool) {
   441  	dist := distuv.Bernoulli{P: p, Src: rand.NewSource(seed)}
   442  	for idx := range out {
   443  		out[idx] = dist.Rand() != float64(0.0)
   444  	}
   445  }
   446  
   447  // fillRandomIsValid populates out with random bools with the probability pctNull of being false using
   448  // seed as the random seed to the generator in order to allow consistency for testing. This uses
   449  // the default Golang random generator distribution of float64 values between 0 and 1 comparing against
   450  // pctNull. If the random value is > pctNull, it is true.
   451  func fillRandomIsValid(seed uint64, pctNull float64, out []bool) {
   452  	r := rand.New(rand.NewSource(seed))
   453  	for idx := range out {
   454  		out[idx] = r.Float64() > pctNull
   455  	}
   456  }
   457  
   458  // InitValues is a convenience function for generating a slice of random values based on the type.
   459  // If the type is parquet.ByteArray or parquet.FixedLenByteArray, heap must not be null.
   460  //
   461  // The default values are:
   462  //
   463  //	[]bool uses the current time as the seed with only values of 1 being false, for use
   464  //	 of creating validity boolean slices.
   465  //	all other types use 0 as the seed
   466  //	a []parquet.ByteArray is populated with lengths between 2 and 12
   467  //	a []parquet.FixedLenByteArray is populated with fixed size random byte arrays of length 12.
   468  func InitValues(values interface{}, heap *memory.Buffer) {
   469  	switch arr := values.(type) {
   470  	case []bool:
   471  		fillRandomIsValid(uint64(time.Now().Unix()), 0.5, arr)
   472  	case []int32:
   473  		FillRandomInt32(0, arr)
   474  	case []int64:
   475  		FillRandomInt64(0, arr)
   476  	case []float32:
   477  		FillRandomFloat32(0, arr)
   478  	case []float64:
   479  		FillRandomFloat64(0, arr)
   480  	case []float16.Num:
   481  		FillRandomFloat16(0, arr)
   482  	case []parquet.Int96:
   483  		FillRandomInt96(0, arr)
   484  	case []parquet.ByteArray:
   485  		FillRandomByteArray(0, arr, heap)
   486  	case []parquet.FixedLenByteArray:
   487  		FillRandomFixedByteArray(0, arr, heap, 12)
   488  	}
   489  }
   490  
   491  // RandomByteArray populates out with random ByteArray values with lengths between minlen and maxlen
   492  // using heap as the actual memory storage used for the bytes generated. Each element of
   493  // out will be some slice of the bytes in heap, and as such heap must outlive the byte array slices.
   494  func RandomByteArray(seed uint64, out []parquet.ByteArray, heap *memory.Buffer, minlen, maxlen int) {
   495  	heap.Resize(len(out) * (maxlen + arrow.Uint32SizeBytes))
   496  
   497  	buf := heap.Bytes()
   498  	r := rand.New(rand.NewSource(seed))
   499  	for idx := range out {
   500  		length := r.Intn(maxlen-minlen+1) + minlen
   501  		r.Read(buf[:length])
   502  		out[idx] = buf[:length]
   503  
   504  		buf = buf[length:]
   505  	}
   506  }
   507  
   508  // RandomDecimals generates n random decimal values with precision determining the byte width
   509  // for the values and seed as the random generator seed to allow consistency for testing. The
   510  // resulting values will be either 32 bytes or 16 bytes each depending on the precision.
   511  func RandomDecimals(n int64, seed uint64, precision int32) []byte {
   512  	r := rand.New(rand.NewSource(seed))
   513  	nreqBytes := pqarrow.DecimalSize(precision)
   514  	byteWidth := 32
   515  	if precision <= 38 {
   516  		byteWidth = 16
   517  	}
   518  
   519  	out := make([]byte, int(int64(byteWidth)*n))
   520  	for i := int64(0); i < n; i++ {
   521  		start := int(i) * byteWidth
   522  		r.Read(out[start : start+int(nreqBytes)])
   523  		// sign extend if the sign bit is set for the last generated byte
   524  		// 0b10000000 == 0x80 == 128
   525  		if out[start+int(nreqBytes)-1]&byte(0x80) != 0 {
   526  			memory.Set(out[start+int(nreqBytes):start+byteWidth], 0xFF)
   527  		}
   528  
   529  		// byte swap for big endian
   530  		if endian.IsBigEndian {
   531  			for j := 0; j+8 <= byteWidth; j += 8 {
   532  				v := binary.LittleEndian.Uint64(out[start+j : start+j+8])
   533  				binary.BigEndian.PutUint64(out[start+j:start+j+8], v)
   534  			}
   535  		}
   536  	}
   537  	return out
   538  }
   539  

View as plain text