// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package bitutil

import (
	
	
	
	

	
	
	
)

// BitmapReader is a simple bitmap reader for a byte slice.
type BitmapReader struct {
	bitmap []byte
	pos    int
	len    int

	current    byte
	byteOffset int
	bitOffset  int
}

// NewBitmapReader creates and returns a new bitmap reader for the given bitmap
func ( []byte, ,  int) *BitmapReader {
	 := byte(0)
	if  > 0 &&  != nil {
		 = [/8]
	}
	return &BitmapReader{
		bitmap:     ,
		byteOffset:  / 8,
		bitOffset:   % 8,
		current:    ,
		len:        ,
	}
}

// Set returns true if the current bit is set
func ( *BitmapReader) () bool {
	return (.current & (1 << .bitOffset)) != 0
}

// NotSet returns true if the current bit is not set
func ( *BitmapReader) () bool {
	return (.current & (1 << .bitOffset)) == 0
}

// Next advances the reader to the next bit in the bitmap.
func ( *BitmapReader) () {
	.bitOffset++
	.pos++
	if .bitOffset == 8 {
		.bitOffset = 0
		.byteOffset++
		if .pos < .len {
			.current = .bitmap[int(.byteOffset)]
		}
	}
}

// Pos returns the current bit position in the bitmap that the reader is looking at
func ( *BitmapReader) () int { return .pos }

// Len returns the total number of bits in the bitmap
func ( *BitmapReader) () int { return .len }

// BitmapWriter is a simple writer for writing bitmaps to byte slices
type BitmapWriter struct {
	buf    []byte
	pos    int
	length int

	curByte    uint8
	bitMask    uint8
	byteOffset int
}

// NewBitmapWriter returns a sequential bitwise writer that preserves surrounding
// bit values as it writes.
func ( []byte, ,  int) *BitmapWriter {
	 := &BitmapWriter{
		buf:        ,
		length:     ,
		byteOffset:  / 8,
		bitMask:    BitMask[%8],
	}
	if  > 0 {
		.curByte = [int(.byteOffset)]
	}
	return 
}

// Reset resets the position and view of the slice to restart writing a bitmap
// to the same byte slice.
func ( *BitmapWriter) (,  int) {
	.pos = 0
	.byteOffset =  / 8
	.bitMask = BitMask[%8]
	.length = 
	if .length > 0 {
		.curByte = .buf[int(.byteOffset)]
	}
}

func ( *BitmapWriter) () int { return .pos }
func ( *BitmapWriter) ()     { .curByte |= .bitMask }
func ( *BitmapWriter) ()   { .curByte &= ^.bitMask }

// Next increments the writer to the next bit for writing.
func ( *BitmapWriter) () {
	.bitMask = .bitMask << 1
	.pos++
	if .bitMask == 0 {
		.bitMask = 0x01
		.buf[.byteOffset] = .curByte
		.byteOffset++
		if .pos < .length {
			.curByte = .buf[int(.byteOffset)]
		}
	}
}

// AppendBools writes a series of booleans to the bitmapwriter and returns
// the number of remaining bytes left in the buffer for writing.
func ( *BitmapWriter) ( []bool) int {
	 := min(.length-.pos, len())
	if  == 0 {
		return 0
	}

	 := bits.TrailingZeros32(uint32(.bitMask))
	// location that the first byte needs to be written to for appending
	 := .buf[int(.byteOffset) : .byteOffset+int(BytesForBits(int64(+)))]
	// update everything but curByte
	[0] = .curByte
	for ,  := range [:] {
		if  {
			SetBit(, +)
		} else {
			ClearBit(, +)
		}
	}

	.pos += 
	.bitMask = BitMask[(+)%8]
	.byteOffset += ( + ) / 8
	.curByte = [len()-1]

	return 
}

// Finish flushes the final byte out to the byteslice in case it was not already
// on a byte aligned boundary.
func ( *BitmapWriter) () {
	if .length > 0 && (.bitMask != 0x01 || .pos < .length) {
		.buf[int(.byteOffset)] = .curByte
	}
}

// BitmapWordReader is a reader for bitmaps that reads a word at a time (a word being an 8 byte uint64)
// and then provides functions to grab the individual trailing bytes after the last word
type BitmapWordReader struct {
	bitmap        []byte
	offset        int
	nwords        int
	trailingBits  int
	trailingBytes int
	curword       uint64
}

// NewBitmapWordReader sets up a word reader, calculates the number of trailing bits and
// number of trailing bytes, along with the number of words.
func ( []byte, ,  int) *BitmapWordReader {
	 :=  % 8
	 :=  / 8
	 := &BitmapWordReader{
		offset: ,
		bitmap: [ : +int(BytesForBits(int64(+)))],
		// decrement wordcount by 1 as we may touch two adjacent words in one iteration
		nwords: /int(unsafe.Sizeof(uint64(0))*8) - 1,
	}
	if .nwords < 0 {
		.nwords = 0
	}
	.trailingBits =  - .nwords*int(unsafe.Sizeof(uint64(0)))*8
	.trailingBytes = int(BytesForBits(int64(.trailingBits)))

	if .nwords > 0 {
		.curword = toFromLEFunc(endian.Native.Uint64(.bitmap))
	} else if  > 0 {
		setLSB(&.curword, .bitmap[0])
	}
	return 
}

// NextWord returns the next full word read from the bitmap, should not be called
// if Words() is 0 as it will step outside of the bounds of the bitmap slice and panic.
//
// We don't perform the bounds checking in order to improve performance.
func ( *BitmapWordReader) () uint64 {
	.bitmap = .bitmap[unsafe.Sizeof(.curword):]
	 := .curword
	 := toFromLEFunc(endian.Native.Uint64(.bitmap))
	if .offset != 0 {
		// combine two adjacent words into one word
		// |<------ next ----->|<---- current ---->|
		// +-------------+-----+-------------+-----+
		// |     ---     |  A  |      B      | --- |
		// +-------------+-----+-------------+-----+
		//                  |         |       offset
		//                  v         v
		//               +-----+-------------+
		//               |  A  |      B      |
		//               +-----+-------------+
		//               |<------ word ----->|
		 >>= uint64(.offset)
		 |=  << (int64(unsafe.Sizeof(uint64(0))*8) - int64(.offset))
	}
	.curword = 
	return 
}

// NextTrailingByte returns the next trailing byte of the bitmap after the last word
// along with the number of valid bits in that byte. When validBits < 8, that
// is the last byte.
//
// If the bitmap ends on a byte alignment, then the last byte can also return 8 valid bits.
// Thus the TrailingBytes function should be used to know how many trailing bytes to read.
func ( *BitmapWordReader) () ( byte,  int) {
	debug.Assert(.trailingBits > 0, "next trailing byte called with no trailing bits")

	if .trailingBits <= 8 {
		// last byte
		 = .trailingBits
		.trailingBits = 0
		 := NewBitmapReader(.bitmap, .offset, )
		for  := 0;  < ; ++ {
			 >>= 1
			if .Set() {
				 |= 0x80
			}
			.Next()
		}
		 >>= (8 - )
		return
	}

	.bitmap = .bitmap[1:]
	 := .bitmap[0]
	 = getLSB(.curword)
	if .offset != 0 {
		 >>= byte(.offset)
		 |=  << (8 - .offset)
	}
	setLSB(&.curword, )
	.trailingBits -= 8
	.trailingBytes--
	 = 8
	return
}

func ( *BitmapWordReader) () int         { return .nwords }
func ( *BitmapWordReader) () int { return .trailingBytes }

// BitmapWordWriter is a bitmap writer for writing a full word at a time (a word being
// a uint64). After the last full word is written, PutNextTrailingByte can be used to
// write the remaining trailing bytes.
type BitmapWordWriter struct {
	bitmap []byte
	offset int
	len    int

	bitMask     uint64
	currentWord uint64
}

// NewBitmapWordWriter initializes a new bitmap word writer which will start writing
// into the byte slice at bit offset start, expecting to write len bits.
func ( []byte, ,  int) *BitmapWordWriter {
	 := &BitmapWordWriter{
		bitmap:  [/8:],
		len:     ,
		offset:   % 8,
		bitMask: (uint64(1) << uint64(%8)) - 1,
	}

	if .offset != 0 {
		if .len >= int(unsafe.Sizeof(uint64(0))*8) {
			.currentWord = toFromLEFunc(endian.Native.Uint64(.bitmap))
		} else if .len > 0 {
			setLSB(&.currentWord, .bitmap[0])
		}
	}
	return 
}

// PutNextWord writes the given word to the bitmap, potentially splitting across
// two adjacent words.
func ( *BitmapWordWriter) ( uint64) {
	 := int(unsafe.Sizeof())
	if .offset != 0 {
		// split one word into two adjacent words, don't touch unused bits
		//               |<------ word ----->|
		//               +-----+-------------+
		//               |  A  |      B      |
		//               +-----+-------------+
		//                  |         |
		//                  v         v       offset
		// +-------------+-----+-------------+-----+
		// |     ---     |  A  |      B      | --- |
		// +-------------+-----+-------------+-----+
		// |<------ next ----->|<---- current ---->|
		 = ( << uint64(.offset)) | ( >> (int64(*8) - int64(.offset)))
		 := toFromLEFunc(endian.Native.Uint64(.bitmap[:]))
		.currentWord = (.currentWord & .bitMask) | ( &^ .bitMask)
		 = ( &^ .bitMask) | ( & .bitMask)
		endian.Native.PutUint64(.bitmap, toFromLEFunc(.currentWord))
		endian.Native.PutUint64(.bitmap[:], toFromLEFunc())
		.currentWord = 
	} else {
		endian.Native.PutUint64(.bitmap, toFromLEFunc())
	}
	.bitmap = .bitmap[:]
}

// PutNextTrailingByte writes the number of bits indicated by validBits from b to
// the bitmap.
func ( *BitmapWordWriter) ( byte,  int) {
	 := getLSB(.currentWord)
	if  == 8 {
		if .offset != 0 {
			 = ( << .offset) | ( >> (8 - .offset))
			 := .bitmap[1]
			 = ( & byte(.bitMask)) | ( &^ byte(.bitMask))
			 = ( &^ byte(.bitMask)) | ( & byte(.bitMask))
			.bitmap[0] = 
			.bitmap[1] = 
			.currentWord = uint64()
		} else {
			.bitmap[0] = 
		}
		.bitmap = .bitmap[1:]
	} else {
		debug.Assert( > 0 &&  < 8, "invalid valid bits in bitmap word writer")
		debug.Assert(BytesForBits(int64(.offset+)) <= int64(len(.bitmap)), "writing trailing byte outside of bounds of bitmap")
		 := NewBitmapWriter(.bitmap, int(.offset), )
		for  := 0;  < ; ++ {
			if &0x01 != 0 {
				.Set()
			} else {
				.Clear()
			}
			.Next()
			 >>= 1
		}
		.Finish()
	}
}

type transferMode int8

const (
	transferCopy transferMode = iota
	transferInvert
)

func transferBitmap( transferMode,  []byte, ,  int,  []byte,  int) {
	if  == 0 {
		// if there's nothing to write, end early.
		return
	}

	 :=  % 8
	 :=  % 8

	// slow path, one of the bitmaps are not byte aligned.
	if  != 0 ||  != 0 {
		 := NewBitmapWordReader(, , )
		 := NewBitmapWordWriter(, , )

		 := .Words()
		for  > 0 {
			--
			if  == transferInvert {
				.PutNextWord(^.NextWord())
			} else {
				.PutNextWord(.NextWord())
			}
		}
		 := .TrailingBytes()
		for  > 0 {
			--
			,  := .NextTrailingByte()
			if  == transferInvert {
				 = ^
			}
			.PutNextTrailingByte(, )
		}
		return
	}

	// fast path, both are starting with byte-aligned bitmaps
	 := int(BytesForBits(int64()))

	// shift by its byte offset
	 = [/8:]
	 = [/8:]

	// Take care of the trailing bits in the last byte
	// E.g., if trailing_bits = 5, last byte should be
	// - low  3 bits: new bits from last byte of data buffer
	// - high 5 bits: old bits from last byte of dest buffer
	 := *8 - 
	 := byte(uint(1)<<(8-)) - 1
	var  byte
	if  == transferInvert {
		for ,  := range [:-1] {
			[] = ^
		}
		 = ^[-1]
	} else {
		copy(, [:-1])
		 = [-1]
	}

	[-1] &= ^
	[-1] |=  & 
}

// CopyBitmap copies the bitmap indicated by src, starting at bit offset srcOffset,
// and copying length bits into dst, starting at bit offset dstOffset.
func ( []byte, ,  int,  []byte,  int) {
	transferBitmap(transferCopy, , , , , )
}

// InvertBitmap copies a bit range of a bitmap, inverting it as it copies
// over into the destination.
func ( []byte, ,  int,  []byte,  int) {
	transferBitmap(transferInvert, , , , , )
}

type bitOp struct {
	opWord    func(uint64, uint64) uint64
	opByte    func(byte, byte) byte
	opAligned func(l, r, o []byte)
}

var (
	bitAndOp = bitOp{
		opWord:    func(,  uint64) uint64 { return  &  },
		opByte:    func(,  byte) byte { return  &  },
		opAligned: alignedBitAndGo,
	}
	bitOrOp = bitOp{
		opWord:    func(,  uint64) uint64 { return  |  },
		opByte:    func(,  byte) byte { return  |  },
		opAligned: alignedBitOrGo,
	}
	bitAndNotOp = bitOp{
		opWord:    func(,  uint64) uint64 { return  &^  },
		opByte:    func(,  byte) byte { return  &^  },
		opAligned: alignedBitAndNotGo,
	}
	bitXorOp = bitOp{
		opWord:    func(,  uint64) uint64 { return  ^  },
		opByte:    func(,  byte) byte { return  ^  },
		opAligned: alignedBitXorGo,
	}
)

func alignedBitmapOp( bitOp, ,  []byte, ,  int64,  []byte,  int64,  int64) {
	debug.Assert(%8 == %8, "aligned bitmap op called with unaligned offsets")
	debug.Assert(%8 == %8, "aligned bitmap op called with unaligned output offset")

	 := BytesForBits( + %8)
	 = [/8:]
	 = [/8:]
	 = [/8:]
	 := ( + %8)
	switch  {
	case 0:
		return
	case 1: // everything within a single byte
		// (length+lOffset%8) <= 8
		 := PrecedingBitmask[%8]
		if  != 0 {
			 |= TrailingBitmask[(+)%8]
		}
		[0] = ([0] & ) | (.opByte([0], [0]) &^ )
	case 2: // don't send zero length to opAligned
		 := PrecedingBitmask[%8]
		[0] = ([0] & ) | (.opByte([0], [0]) &^ )
		 := byte(0)
		if  != 0 {
			 = TrailingBitmask[(+)%8]
		}
		[1] = ([1] & ) | (.opByte([1], [1]) &^ )
	default:
		 := PrecedingBitmask[%8]
		[0] = ([0] & ) | (.opByte([0], [0]) &^ )

		.opAligned([1:-1], [1:-1], [1:-1])

		 := byte(0)
		if  != 0 {
			 = TrailingBitmask[(+)%8]
		}
		[-1] = ([-1] & ) | (.opByte([-1], [-1]) &^ )
	}
}

func unalignedBitmapOp( bitOp, ,  []byte, ,  int64,  []byte,  int64,  int64) {
	 := NewBitmapWordReader(, int(), int())
	 := NewBitmapWordReader(, int(), int())
	 := NewBitmapWordWriter(, int(), int())

	for  := .Words();  > 0; -- {
		.PutNextWord(.opWord(.NextWord(), .NextWord()))
	}
	for  := .TrailingBytes();  > 0; -- {
		,  := .NextTrailingByte()
		,  := .NextTrailingByte()
		debug.Assert( == , "unexpected mismatch of valid bits")
		.PutNextTrailingByte(.opByte(, ), )
	}
}

func ( bitOp, ,  []byte, ,  int64,  []byte, ,  int64) {
	if (%8 == %8) && (%8 == %8) {
		// fastcase!
		alignedBitmapOp(, , , , , , , )
	} else {
		unalignedBitmapOp(, , , , , , , )
	}
}

func ( memory.Allocator,  bitOp, ,  []byte, ,  int64,  int64,  int64) *memory.Buffer {
	 :=  + 
	 := memory.NewResizableBuffer()
	.Resize(int(BytesForBits()))
	BitmapOp(, , , , , .Bytes(), , )
	return 
}

func (,  []byte, ,  int64,  []byte,  int64,  int64) {
	BitmapOp(bitAndOp, , , , , , , )
}

func (,  []byte, ,  int64,  []byte,  int64,  int64) {
	BitmapOp(bitOrOp, , , , , , , )
}

func ( memory.Allocator, ,  []byte, ,  int64, ,  int64) *memory.Buffer {
	return BitmapOpAlloc(, bitAndOp, , , , , , )
}

func ( memory.Allocator, ,  []byte, ,  int64, ,  int64) *memory.Buffer {
	return BitmapOpAlloc(, bitOrOp, , , , , , )
}

func (,  []byte, ,  int64,  []byte,  int64,  int64) {
	BitmapOp(bitAndNotOp, , , , , , , )
}

func ( memory.Allocator, ,  []byte, ,  int64, ,  int64) *memory.Buffer {
	return BitmapOpAlloc(, bitAndNotOp, , , , , , )
}

func (,  []byte, ,  int64,  []byte,  int64,  int64) {
	BitmapOp(bitXorOp, , , , , , , )
}

func ( memory.Allocator, ,  []byte, ,  int64, ,  int64) *memory.Buffer {
	return BitmapOpAlloc(, bitXorOp, , , , , , )
}

func (,  []byte, ,  int64,  int64) bool {
	if %8 == 0 && %8 == 0 {
		// byte aligned, fast path, can use bytes.Equal (memcmp)
		 :=  / 8
		 :=  / 8
		 :=  / 8
		if !bytes.Equal([:+], [:+]) {
			return false
		}

		// check trailing bits
		for  := ( / 8) * 8;  < ; ++ {
			if BitIsSet(, int(+)) != BitIsSet(, int(+)) {
				return false
			}
		}
		return true
	}

	 := NewBitmapWordReader(, int(), int())
	 := NewBitmapWordReader(, int(), int())

	 := .Words()
	for  > 0 {
		--
		if .NextWord() != .NextWord() {
			return false
		}
	}

	 := .TrailingBytes()
	for  > 0 {
		--
		,  := .NextTrailingByte()
		,  := .NextTrailingByte()
		if  !=  {
			return false
		}
	}
	return true
}

// OptionalBitIndexer is a convenience wrapper for getting bits from
// a bitmap which may or may not be nil.
type OptionalBitIndexer struct {
	Bitmap []byte
	Offset int
}

func ( *OptionalBitIndexer) ( int) bool {
	return .Bitmap == nil || BitIsSet(.Bitmap, .Offset+)
}

type Bitmap struct {
	Data        []byte
	Offset, Len int64
}

func bitLength( []Bitmap) (int64, error) {
	for ,  := range [1:] {
		if .Len != [0].Len {
			return -1, errors.New("bitmaps must be same length")
		}
	}
	return [0].Len, nil
}

func runVisitWordsAndWriteLoop( int64,  []*BitmapWordReader,  []*BitmapWordWriter,  func(,  []uint64)) {
	const  int64 = int64(uint64SizeBits)

	 := make([]uint64, len())
	 := make([]uint64, len())

	// every reader will have same number of words, since they are same
	// length'ed. This will be inefficient in some cases. When there's
	// offsets beyond the Word boundary, every word would have to be
	// created from 2 adjoining words
	 := int64([0].Words())
	 -=  * 
	for  > 0 {
		--
		for  := range  {
			[] = [].NextWord()
		}
		(, )
		for  := range  {
			[].PutNextWord([])
		}
	}

	// every reader will have the same number of trailing bytes, because
	// we already confirmed they have the same length. Because
	// offsets beyond the Word boundary can cause adjoining words, the
	// tailing portion could be more than one word remaining full/partial
	// words to write.
	if  == 0 {
		return
	}

	// convert the word visitor to a bytevisitor
	 := func(,  []byte) {
		for ,  := range  {
			[] = uint64()
		}
		(, )
		for ,  := range  {
			[] = byte()
		}
	}

	 := make([]byte, len())
	 := make([]byte, len())
	 := [0].trailingBytes
	for  > 0 {
		--
		memory.Set(, 0)
		memory.Set(, 0)

		var  int
		for  := range  {
			[],  = [].NextTrailingByte()
		}
		(, )
		for ,  := range  {
			[].PutNextTrailingByte(, )
		}
	}
}

// VisitWordsAndWrite visits words of bits from each input bitmap and
// collects outputs to a slice of output Bitmaps.
//
// All bitmaps must have identical lengths. The first bit in a visited
// bitmap may be offset within the first visited word, but words will
// otherwise contain densely packed bits loaded from the bitmap. That
// offset within the first word is returned.
//
// NOTE: this function is efficient on 3+ sufficiently large bitmaps.
// It also has a large prolog/epilog overhead and should be used
// carefully in other cases. For 2 or fewer bitmaps, and/or smaller
// bitmaps, try BitmapReader and or other utilities.
func ( []Bitmap,  []Bitmap,  func(,  []uint64)) error {
	,  := bitLength()
	if  != nil {
		return 
	}

	,  := make([]*BitmapWordReader, len()), make([]*BitmapWordWriter, len())
	for ,  := range  {
		[] = NewBitmapWordReader(.Data, int(.Offset), int(.Len))
	}
	for ,  := range  {
		[] = NewBitmapWordWriter(.Data, int(.Offset), int(.Len))
	}
	runVisitWordsAndWriteLoop(, , , )
	return nil
}