...

Text file src/github.com/apache/arrow/go/v15/internal/hashing/xxh3_memo_table.gen.go.tmpl

Documentation: github.com/apache/arrow/go/v15/internal/hashing

     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
    17package hashing
    18
    19import (
    20  "github.com/apache/arrow/go/v15/arrow/bitutil"  
    21  "github.com/apache/arrow/go/v15/internal/utils"  
    22)
    23
    24{{range .In}}
    25type payload{{.Name}} struct {
    26	val     {{.name}}
    27	memoIdx int32
    28}
    29
    30type entry{{.Name}} struct {
    31	h       uint64
    32	payload payload{{.Name}}
    33}
    34
    35func (e entry{{.Name}}) Valid() bool { return e.h != sentinel }
    36
    37// {{.Name}}HashTable is a hashtable specifically for {{.name}} that
    38// is utilized with the MemoTable to generalize interactions for easier
    39// implementation of dictionaries without losing performance.
    40type {{.Name}}HashTable struct {
    41	cap     uint64
    42	capMask uint64
    43	size    uint64
    44
    45	entries []entry{{.Name}}
    46}
    47
    48// New{{.Name}}HashTable returns a new hash table for {{.name}} values
    49// initialized with the passed in capacity or 32 whichever is larger.
    50func New{{.Name}}HashTable(cap uint64) *{{.Name}}HashTable {
    51	initCap := uint64(bitutil.NextPowerOf2(int(max(cap, 32))))
    52	ret := &{{.Name}}HashTable{cap: initCap, capMask: initCap - 1, size: 0}
    53	ret.entries = make([]entry{{.Name}}, initCap)
    54	return ret
    55}
    56
    57// Reset drops all of the values in this hash table and re-initializes it
    58// with the specified initial capacity as if by calling New, but without having
    59// to reallocate the object.
    60func (h *{{.Name}}HashTable) Reset(cap uint64) {
    61	h.cap = uint64(bitutil.NextPowerOf2(int(max(cap, 32))))
    62	h.capMask = h.cap - 1
    63	h.size = 0
    64	h.entries = make([]entry{{.Name}}, h.cap)
    65}
    66
    67// CopyValues is used for copying the values out of the hash table into the
    68// passed in slice, in the order that they were first inserted
    69func (h *{{.Name}}HashTable) CopyValues(out []{{.name}}) {
    70  h.CopyValuesSubset(0, out)
    71}
    72
    73// CopyValuesSubset copies a subset of the values in the hashtable out, starting
    74// with the value at start, in the order that they were inserted.
    75func (h *{{.Name}}HashTable) CopyValuesSubset(start int, out []{{.name}}) {
    76  h.VisitEntries(func(e *entry{{.Name}}) {
    77    idx := e.payload.memoIdx - int32(start)
    78    if idx >= 0 {
    79      out[idx] = e.payload.val
    80    }
    81  })
    82}
    83
    84func (h *{{.Name}}HashTable) WriteOut(out []byte) {
    85  h.WriteOutSubset(0, out)
    86}
    87
    88func (h *{{.Name}}HashTable) WriteOutSubset(start int, out []byte) {
    89  data := arrow.{{.Name}}Traits.CastFromBytes(out)
    90  h.VisitEntries(func(e *entry{{.Name}}) {
    91    idx := e.payload.memoIdx - int32(start)
    92    if idx >= 0 {
    93{{if and (ne .Name "Int8") (ne .Name "Uint8") -}}    
    94      data[idx] = utils.ToLE{{.Name}}(e.payload.val)
    95{{else -}}
    96      data[idx] = e.payload.val
    97{{end -}}
    98    }
    99  })
   100}
   101
   102func (h *{{.Name}}HashTable) needUpsize() bool { return h.size*uint64(loadFactor) >= h.cap }
   103
   104func ({{.Name}}HashTable) fixHash(v uint64) uint64 {
   105	if v == sentinel {
   106		return 42
   107	}
   108	return v
   109}
   110
   111// Lookup retrieves the entry for a given hash value assuming it's payload value returns
   112// true when passed to the cmp func. Returns a pointer to the entry for the given hash value,
   113// and a boolean as to whether it was found. It is not safe to use the pointer if the bool is false.
   114func (h *{{.Name}}HashTable) Lookup(v uint64, cmp func({{.name}}) bool) (*entry{{.Name}}, bool) {
   115	idx, ok := h.lookup(v, h.capMask, cmp)
   116	return &h.entries[idx], ok
   117}
   118
   119func (h *{{.Name}}HashTable) lookup(v uint64, szMask uint64, cmp func({{.name}}) bool) (uint64, bool) {
   120	const perturbShift uint8 = 5
   121
   122	var (
   123		idx     uint64
   124		perturb uint64
   125		e       *entry{{.Name}}
   126	)
   127
   128	v = h.fixHash(v)
   129	idx = v & szMask
   130	perturb = (v >> uint64(perturbShift)) + 1
   131
   132	for {
   133		e = &h.entries[idx]
   134		if e.h == v && cmp(e.payload.val) {
   135			return idx, true
   136		}
   137
   138		if e.h == sentinel {
   139			return idx, false
   140		}
   141
   142		// perturbation logic inspired from CPython's set/dict object
   143		// the goal is that all 64 bits of unmasked hash value eventually
   144		// participate int he probing sequence, to minimize clustering
   145		idx = (idx + perturb) & szMask
   146		perturb = (perturb >> uint64(perturbShift)) + 1
   147	}
   148}
   149
   150func (h *{{.Name}}HashTable) upsize(newcap uint64) error {
   151	newMask := newcap - 1
   152
   153	oldEntries := h.entries
   154	h.entries = make([]entry{{.Name}}, newcap)
   155	for _, e := range oldEntries {
   156		if e.Valid() {
   157			idx, _ := h.lookup(e.h, newMask, func({{.name}}) bool { return false })
   158			h.entries[idx] = e
   159		}
   160	}
   161	h.cap = newcap
   162	h.capMask = newMask
   163	return nil
   164}
   165
   166// Insert updates the given entry with the provided hash value, payload value and memo index.
   167// The entry pointer must have been retrieved via lookup in order to actually insert properly.
   168func (h *{{.Name}}HashTable) Insert(e *entry{{.Name}}, v uint64, val {{.name}}, memoIdx int32) error {
   169	e.h = h.fixHash(v)
   170	e.payload.val = val
   171	e.payload.memoIdx = memoIdx
   172	h.size++
   173
   174	if h.needUpsize() {
   175		h.upsize(h.cap * uint64(loadFactor) * 2)
   176	}
   177	return nil
   178}
   179
   180// VisitEntries will call the passed in function on each *valid* entry in the hash table,
   181// a valid entry being one which has had a value inserted into it.
   182func (h *{{.Name}}HashTable) VisitEntries(visit func(*entry{{.Name}})) {
   183	for _, e := range h.entries {
   184		if e.Valid() {
   185			visit(&e)
   186		}
   187	}
   188}
   189
   190// {{.Name}}MemoTable is a wrapper over the appropriate hashtable to provide an interface
   191// conforming to the MemoTable interface defined in the encoding package for general interactions
   192// regarding dictionaries.
   193type {{.Name}}MemoTable struct {
   194  tbl *{{.Name}}HashTable
   195  nullIdx int32
   196}
   197
   198// New{{.Name}}MemoTable returns a new memotable with num entries pre-allocated to reduce further
   199// allocations when inserting.
   200func New{{.Name}}MemoTable(num int64) *{{.Name}}MemoTable {
   201  return &{{.Name}}MemoTable{tbl: New{{.Name}}HashTable(uint64(num)), nullIdx: KeyNotFound}
   202}
   203
   204func ({{.Name}}MemoTable) TypeTraits() TypeTraits {
   205  return arrow.{{.Name}}Traits
   206}
   207
   208// Reset allows this table to be re-used by dumping all the data currently in the table.
   209func (s *{{.Name}}MemoTable) Reset() {
   210  s.tbl.Reset(32)
   211  s.nullIdx = KeyNotFound
   212}
   213
   214// Size returns the current number of inserted elements into the table including if a null
   215// has been inserted.
   216func (s *{{.Name}}MemoTable) Size() int {
   217  sz := int(s.tbl.size)
   218  if _, ok := s.GetNull(); ok {
   219    sz++
   220  }
   221  return sz
   222}
   223
   224// GetNull returns the index of an inserted null or KeyNotFound along with a bool
   225// that will be true if found and false if not.
   226func (s *{{.Name}}MemoTable) GetNull() (int, bool) {
   227  return int(s.nullIdx), s.nullIdx != KeyNotFound
   228}
   229
   230// GetOrInsertNull will return the index of the null entry or insert a null entry
   231// if one currently doesn't exist. The found value will be true if there was already
   232// a null in the table, and false if it inserted one.
   233func (s *{{.Name}}MemoTable) GetOrInsertNull() (idx int, found bool) {
   234  idx, found = s.GetNull()
   235  if !found {
   236    idx = s.Size()
   237    s.nullIdx = int32(idx)
   238  }
   239  return
   240}
   241
   242// CopyValues will copy the values from the memo table out into the passed in slice
   243// which must be of the appropriate type.
   244func (s *{{.Name}}MemoTable) CopyValues(out interface{}) {
   245  s.CopyValuesSubset(0, out)
   246}
   247
   248// CopyValuesSubset is like CopyValues but only copies a subset of values starting
   249// at the provided start index
   250func (s *{{.Name}}MemoTable) CopyValuesSubset(start int, out interface{}) {
   251  s.tbl.CopyValuesSubset(start, out.([]{{.name}}))
   252}
   253
   254func (s *{{.Name}}MemoTable) WriteOut(out []byte) {
   255  s.tbl.CopyValues(arrow.{{.Name}}Traits.CastFromBytes(out))
   256}
   257
   258func (s *{{.Name}}MemoTable) WriteOutSubset(start int, out []byte) {
   259  s.tbl.CopyValuesSubset(start, arrow.{{.Name}}Traits.CastFromBytes(out))
   260}
   261
   262func (s *{{.Name}}MemoTable) WriteOutLE(out []byte) {
   263  s.tbl.WriteOut(out)
   264}
   265
   266func (s *{{.Name}}MemoTable) WriteOutSubsetLE(start int, out []byte) {
   267  s.tbl.WriteOutSubset(start, out)
   268}
   269
   270// Get returns the index of the requested value in the hash table or KeyNotFound
   271// along with a boolean indicating if it was found or not.
   272func (s *{{.Name}}MemoTable) Get(val interface{}) (int, bool) {
   273{{if and (ne .Name "Float32") (ne .Name "Float64") }}
   274  h := hashInt(uint64(val.({{.name}})), 0)
   275  if e, ok := s.tbl.Lookup(h, func(v {{.name}}) bool { return val.({{.name}}) == v }); ok {
   276{{ else -}}
   277  var cmp func({{.name}}) bool
   278  {{if eq .Name "Float32"}}
   279  if math.IsNaN(float64(val.(float32))) {
   280    cmp = isNan32Cmp
   281    // use consistent internal bit pattern for NaN regardless of the pattern
   282    // that is passed to us. NaN is NaN is NaN
   283    val = float32(math.NaN())
   284  {{ else -}}
   285  if math.IsNaN(val.(float64)) {
   286    cmp = math.IsNaN
   287    // use consistent internal bit pattern for NaN regardless of the pattern
   288    // that is passed to us. NaN is NaN is NaN
   289    val = math.NaN()
   290  {{end -}}
   291  } else {
   292    cmp = func(v {{.name}}) bool { return val.({{.name}}) == v }
   293  }
   294
   295  h := hash{{.Name}}(val.({{.name}}), 0)  
   296  if e, ok := s.tbl.Lookup(h, cmp); ok {
   297{{ end -}}
   298    return int(e.payload.memoIdx), ok
   299  }
   300  return KeyNotFound, false
   301}
   302
   303// GetOrInsert will return the index of the specified value in the table, or insert the
   304// value into the table and return the new index. found indicates whether or not it already
   305// existed in the table (true) or was inserted by this call (false).
   306func (s *{{.Name}}MemoTable) GetOrInsert(val interface{}) (idx int, found bool, err error) {
   307  {{if and (ne .Name "Float32") (ne .Name "Float64") }}
   308  h := hashInt(uint64(val.({{.name}})), 0)
   309  e, ok := s.tbl.Lookup(h, func(v {{.name}}) bool {
   310    return val.({{.name}}) == v
   311  })
   312{{ else }}  
   313  var cmp func({{.name}}) bool
   314  {{if eq .Name "Float32"}}
   315  if math.IsNaN(float64(val.(float32))) {
   316    cmp = isNan32Cmp
   317    // use consistent internal bit pattern for NaN regardless of the pattern
   318    // that is passed to us. NaN is NaN is NaN
   319    val = float32(math.NaN()) 
   320  {{ else -}}
   321  if math.IsNaN(val.(float64)) {  
   322    cmp = math.IsNaN
   323    // use consistent internal bit pattern for NaN regardless of the pattern
   324    // that is passed to us. NaN is NaN is NaN
   325    val = math.NaN()
   326  {{end -}}
   327  } else {
   328    cmp = func(v {{.name}}) bool { return val.({{.name}}) == v }
   329  }
   330  
   331  h := hash{{.Name}}(val.({{.name}}), 0)
   332  e, ok := s.tbl.Lookup(h, cmp)
   333{{ end }}
   334  if ok {
   335    idx = int(e.payload.memoIdx)
   336    found = true
   337  } else {
   338    idx = s.Size()
   339    s.tbl.Insert(e, h, val.({{.name}}), int32(idx))
   340  }
   341  return
   342}
   343
   344
   345// GetOrInsertBytes is unimplemented
   346func (s *{{.Name}}MemoTable) GetOrInsertBytes(val []byte) (idx int, found bool, err error) {
   347    panic("unimplemented")
   348}
   349{{end}}

View as plain text